From 6376b447c9b18dc9f0b72dc8dba46d78ec655049 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 15 Oct 2024 02:59:10 -0700 Subject: [PATCH 001/313] Spark 3.3, 3.4, 3.5: Remove unnecessary copying of FileScanTask (#11319) --- .../org/apache/iceberg/spark/source/SparkScanBuilder.java | 4 +--- .../org/apache/iceberg/spark/source/SparkScanBuilder.java | 4 +--- .../org/apache/iceberg/spark/source/SparkScanBuilder.java | 4 +--- 3 files changed, 3 insertions(+), 9 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index afb0f434aa41..5634e1436081 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -48,7 +48,6 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; @@ -243,8 +242,7 @@ public boolean pushAggregation(Aggregation aggregation) { scan = scan.filter(filterExpression()); try (CloseableIterable fileScanTasks = scan.planFiles()) { - List tasks = ImmutableList.copyOf(fileScanTasks); - for (FileScanTask task : tasks) { + for (FileScanTask task : fileScanTasks) { if (!task.deletes().isEmpty()) { LOG.info("Skipping aggregate pushdown: detected row level deletes"); return false; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 9dc214a755d3..d511fefd8ae0 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -49,7 +49,6 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; @@ -239,8 +238,7 @@ public boolean pushAggregation(Aggregation aggregation) { buildIcebergBatchScan(true /* include Column Stats */, schemaWithMetadataColumns()); try (CloseableIterable fileScanTasks = scan.planFiles()) { - List tasks = ImmutableList.copyOf(fileScanTasks); - for (FileScanTask task : tasks) { + for (FileScanTask task : fileScanTasks) { if (!task.deletes().isEmpty()) { LOG.info("Skipping aggregate pushdown: detected row level deletes"); return false; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 9dc214a755d3..d511fefd8ae0 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -49,7 +49,6 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; @@ -239,8 +238,7 @@ public boolean pushAggregation(Aggregation aggregation) { buildIcebergBatchScan(true /* include Column Stats */, schemaWithMetadataColumns()); try (CloseableIterable fileScanTasks = scan.planFiles()) { - List tasks = ImmutableList.copyOf(fileScanTasks); - for (FileScanTask task : tasks) { + for (FileScanTask task : fileScanTasks) { if (!task.deletes().isEmpty()) { LOG.info("Skipping aggregate pushdown: detected row level deletes"); return false; From 33b33f3ec835ce69025aa827b5b5ba6e2fbd98dd Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 15 Oct 2024 20:02:49 +0200 Subject: [PATCH 002/313] Core: Rename DeleteFileHolder to PendingDeleteFile / Optimize duplicate data/delete file detection (#11254) --- .../apache/iceberg/ManifestFilterManager.java | 2 +- .../iceberg/MergingSnapshotProducer.java | 69 ++++---- .../org/apache/iceberg/SnapshotProducer.java | 153 ++++++++++++++++-- 3 files changed, 178 insertions(+), 46 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index fddb1a161637..adc5e1b7d3c2 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -68,9 +68,9 @@ public String partition() { private final Map specsById; private final PartitionSet deleteFilePartitions; + private final Set deleteFiles = newFileSet(); private final PartitionSet dropPartitions; private final CharSequenceSet deletePaths = CharSequenceSet.empty(); - private final Set deleteFiles = newFileSet(); private Expression deleteExpression = Expressions.alwaysFalse(); private long minSequenceNumber = 0; private boolean failAnyDelete = false; diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 2209b348227d..ab55f86ebf6f 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -30,7 +30,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.stream.Collectors; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.exceptions.ValidationException; @@ -42,7 +41,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Predicate; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Iterators; @@ -82,11 +80,9 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final ManifestFilterManager deleteFilterManager; // update data - private final Map> newDataFilesBySpec = Maps.newHashMap(); - private final DataFileSet newDataFiles = DataFileSet.create(); - private final DeleteFileSet newDeleteFiles = DeleteFileSet.create(); + private final Map newDataFilesBySpec = Maps.newHashMap(); private Long newDataFilesDataSequenceNumber; - private final Map> newDeleteFilesBySpec = Maps.newHashMap(); + private final Map newDeleteFilesBySpec = Maps.newHashMap(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); private final SnapshotSummary.Builder addedFilesSummary = SnapshotSummary.builder(); @@ -161,12 +157,9 @@ protected Expression rowFilter() { } protected List addedDataFiles() { - return ImmutableList.copyOf( - newDataFilesBySpec.values().stream().flatMap(List::stream).collect(Collectors.toList())); - } - - protected Map> addedDataFilesBySpec() { - return ImmutableMap.copyOf(newDataFilesBySpec); + return newDataFilesBySpec.values().stream() + .flatMap(Set::stream) + .collect(ImmutableList.toImmutableList()); } protected void failAnyDelete() { @@ -236,43 +229,49 @@ protected boolean addsDeleteFiles() { /** Add a data file to the new snapshot. */ protected void add(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); - if (newDataFiles.add(file)) { - PartitionSpec fileSpec = ops.current().spec(file.specId()); - Preconditions.checkArgument( - fileSpec != null, - "Cannot find partition spec %s for data file: %s", - file.specId(), - file.path()); - - addedFilesSummary.addedFile(fileSpec, file); + PartitionSpec spec = spec(file.specId()); + Preconditions.checkArgument( + spec != null, + "Cannot find partition spec %s for data file: %s", + file.specId(), + file.location()); + + DataFileSet dataFiles = + newDataFilesBySpec.computeIfAbsent(spec, ignored -> DataFileSet.create()); + if (dataFiles.add(file)) { + addedFilesSummary.addedFile(spec, file); hasNewDataFiles = true; - List dataFiles = - newDataFilesBySpec.computeIfAbsent(fileSpec, ignored -> Lists.newArrayList()); - dataFiles.add(file); } } + private PartitionSpec spec(int specId) { + return ops.current().spec(specId); + } + /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file) { Preconditions.checkNotNull(file, "Invalid delete file: null"); - add(new DeleteFileHolder(file)); + add(new PendingDeleteFile(file)); } /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file, long dataSequenceNumber) { Preconditions.checkNotNull(file, "Invalid delete file: null"); - add(new DeleteFileHolder(file, dataSequenceNumber)); + add(new PendingDeleteFile(file, dataSequenceNumber)); } - private void add(DeleteFileHolder fileHolder) { - int specId = fileHolder.deleteFile().specId(); - PartitionSpec fileSpec = ops.current().spec(specId); - List deleteFiles = - newDeleteFilesBySpec.computeIfAbsent(specId, s -> Lists.newArrayList()); - - if (newDeleteFiles.add(fileHolder.deleteFile())) { - deleteFiles.add(fileHolder); - addedFilesSummary.addedFile(fileSpec, fileHolder.deleteFile()); + private void add(PendingDeleteFile file) { + PartitionSpec spec = spec(file.specId()); + Preconditions.checkArgument( + spec != null, + "Cannot find partition spec %s for delete file: %s", + file.specId(), + file.location()); + + DeleteFileSet deleteFiles = + newDeleteFilesBySpec.computeIfAbsent(spec.specId(), ignored -> DeleteFileSet.create()); + if (deleteFiles.add(file)) { + addedFilesSummary.addedFile(spec, file); hasNewDeleteFiles = true; } } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 27724f787dd2..33114baa641d 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -35,6 +35,7 @@ import com.github.benmanes.caffeine.cache.LoadingCache; import java.io.IOException; import java.math.RoundingMode; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -595,20 +596,22 @@ private List writeDataFileGroup( } protected List writeDeleteManifests( - Collection files, PartitionSpec spec) { + Collection files, PartitionSpec spec) { return writeManifests(files, group -> writeDeleteFileGroup(group, spec)); } private List writeDeleteFileGroup( - Collection files, PartitionSpec spec) { + Collection files, PartitionSpec spec) { RollingManifestWriter writer = newRollingDeleteManifestWriter(spec); try (RollingManifestWriter closableWriter = writer) { - for (DeleteFileHolder file : files) { + for (DeleteFile file : files) { + Preconditions.checkArgument( + file instanceof PendingDeleteFile, "Invalid delete file: must be PendingDeleteFile"); if (file.dataSequenceNumber() != null) { - closableWriter.add(file.deleteFile(), file.dataSequenceNumber()); + closableWriter.add(file, file.dataSequenceNumber()); } else { - closableWriter.add(file.deleteFile()); + closableWriter.add(file); } } } catch (IOException e) { @@ -752,7 +755,7 @@ private static void updateTotal( } } - protected static class DeleteFileHolder { + protected static class PendingDeleteFile implements DeleteFile { private final DeleteFile deleteFile; private final Long dataSequenceNumber; @@ -762,7 +765,7 @@ protected static class DeleteFileHolder { * @param deleteFile delete file * @param dataSequenceNumber data sequence number to apply */ - DeleteFileHolder(DeleteFile deleteFile, long dataSequenceNumber) { + PendingDeleteFile(DeleteFile deleteFile, long dataSequenceNumber) { this.deleteFile = deleteFile; this.dataSequenceNumber = dataSequenceNumber; } @@ -772,17 +775,147 @@ protected static class DeleteFileHolder { * * @param deleteFile delete file */ - DeleteFileHolder(DeleteFile deleteFile) { + PendingDeleteFile(DeleteFile deleteFile) { this.deleteFile = deleteFile; this.dataSequenceNumber = null; } - public DeleteFile deleteFile() { - return deleteFile; + private PendingDeleteFile wrap(DeleteFile file) { + if (null != dataSequenceNumber) { + return new PendingDeleteFile(file, dataSequenceNumber); + } + + return new PendingDeleteFile(file); } + @Override public Long dataSequenceNumber() { return dataSequenceNumber; } + + @Override + public Long fileSequenceNumber() { + return deleteFile.fileSequenceNumber(); + } + + @Override + public DeleteFile copy() { + return wrap(deleteFile.copy()); + } + + @Override + public DeleteFile copyWithoutStats() { + return wrap(deleteFile.copyWithoutStats()); + } + + @Override + public DeleteFile copyWithStats(Set requestedColumnIds) { + return wrap(deleteFile.copyWithStats(requestedColumnIds)); + } + + @Override + public DeleteFile copy(boolean withStats) { + return wrap(deleteFile.copy(withStats)); + } + + @Override + public String manifestLocation() { + return deleteFile.manifestLocation(); + } + + @Override + public Long pos() { + return deleteFile.pos(); + } + + @Override + public int specId() { + return deleteFile.specId(); + } + + @Override + public FileContent content() { + return deleteFile.content(); + } + + @Override + public CharSequence path() { + return deleteFile.path(); + } + + @Override + public String location() { + return deleteFile.location(); + } + + @Override + public FileFormat format() { + return deleteFile.format(); + } + + @Override + public StructLike partition() { + return deleteFile.partition(); + } + + @Override + public long recordCount() { + return deleteFile.recordCount(); + } + + @Override + public long fileSizeInBytes() { + return deleteFile.fileSizeInBytes(); + } + + @Override + public Map columnSizes() { + return deleteFile.columnSizes(); + } + + @Override + public Map valueCounts() { + return deleteFile.valueCounts(); + } + + @Override + public Map nullValueCounts() { + return deleteFile.nullValueCounts(); + } + + @Override + public Map nanValueCounts() { + return deleteFile.nanValueCounts(); + } + + @Override + public Map lowerBounds() { + return deleteFile.lowerBounds(); + } + + @Override + public Map upperBounds() { + return deleteFile.upperBounds(); + } + + @Override + public ByteBuffer keyMetadata() { + return deleteFile.keyMetadata(); + } + + @Override + public List splitOffsets() { + return deleteFile.splitOffsets(); + } + + @Override + public List equalityFieldIds() { + return deleteFile.equalityFieldIds(); + } + + @Override + public Integer sortOrderId() { + return deleteFile.sortOrderId(); + } } } From 32b1ab6ea833f4067b92f7943cbfc0057faba4c7 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 16 Oct 2024 08:46:16 +0200 Subject: [PATCH 003/313] Core: Fix version number in deprecation note for invalidateAll (#11325) --- core/src/main/java/org/apache/iceberg/io/ContentCache.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/io/ContentCache.java b/core/src/main/java/org/apache/iceberg/io/ContentCache.java index 5757c542dc09..fd171f7806e2 100644 --- a/core/src/main/java/org/apache/iceberg/io/ContentCache.java +++ b/core/src/main/java/org/apache/iceberg/io/ContentCache.java @@ -140,7 +140,7 @@ public void invalidate(String key) { } /** - * @deprecated since 1.6.0, will be removed in 1.7.0; This method does only best-effort + * @deprecated since 1.7.0, will be removed in 2.0.0; This method does only best-effort * invalidation and is susceptible to a race condition. If the caller changed the state that * could be cached (perhaps files on the storage) and calls this method, there is no guarantee * that the cache will not contain stale entries some time after this method returns. From 5e279c868f8e2087f88ae779d8cc8474768bc5e2 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 16 Oct 2024 21:32:51 +0900 Subject: [PATCH 004/313] Build, Spark, Flink: Bump junit from 5.10.1 to 5.11.1 (#11262) --- .../iceberg/ParameterizedTestExtension.java | 11 +++++---- .../sink/TestFlinkIcebergSinkV2Branch.java | 17 ++++++++++---- .../flink/source/TestFlinkSourceSql.java | 2 ++ .../flink/source/TestIcebergSourceSql.java | 2 ++ .../sink/TestFlinkIcebergSinkV2Branch.java | 17 ++++++++++---- .../flink/sink/TestIcebergSinkV2Branch.java | 11 +-------- .../flink/source/TestFlinkSourceSql.java | 2 ++ .../flink/source/TestIcebergSourceSql.java | 2 ++ .../sink/TestFlinkIcebergSinkV2Branch.java | 17 ++++++++++---- .../flink/sink/TestIcebergSinkV2Branch.java | 11 +-------- .../flink/source/TestFlinkSourceSql.java | 2 ++ .../flink/source/TestIcebergSourceSql.java | 2 ++ gradle/libs.versions.toml | 2 +- .../mr/TestInputFormatReaderDeletes.java | 23 ++++++++----------- .../spark/source/TestSparkReaderDeletes.java | 19 ++++++++------- .../spark/source/TestSparkReaderDeletes.java | 19 ++++++++------- .../spark/extensions/ExtensionsTestBase.java | 2 ++ .../spark/source/TestSparkReaderDeletes.java | 19 ++++++++------- .../spark/sql/TestAggregatePushDown.java | 2 ++ .../sql/TestUnpartitionedWritesToBranch.java | 2 ++ 20 files changed, 103 insertions(+), 81 deletions(-) diff --git a/api/src/test/java/org/apache/iceberg/ParameterizedTestExtension.java b/api/src/test/java/org/apache/iceberg/ParameterizedTestExtension.java index 59652bab9851..2f625092ff7c 100644 --- a/api/src/test/java/org/apache/iceberg/ParameterizedTestExtension.java +++ b/api/src/test/java/org/apache/iceberg/ParameterizedTestExtension.java @@ -73,13 +73,10 @@ public Stream provideTestTemplateInvocationContex // Search method annotated with @Parameters final List parameterProviders = AnnotationSupport.findAnnotatedMethods( - context.getRequiredTestClass(), Parameters.class, HierarchyTraversalMode.TOP_DOWN); + context.getRequiredTestClass(), Parameters.class, HierarchyTraversalMode.BOTTOM_UP); if (parameterProviders.isEmpty()) { throw new IllegalStateException("Cannot find any parameter provider"); } - if (parameterProviders.size() > 1) { - throw new IllegalStateException("Multiple parameter providers are found"); - } Method parameterProvider = parameterProviders.get(0); // Get potential test name @@ -225,7 +222,11 @@ private Stream createContextForParameters( Stream parameterValueStream, String testNameTemplate, ExtensionContext context) { // Search fields annotated by @Parameter final List parameterFields = - AnnotationSupport.findAnnotatedFields(context.getRequiredTestClass(), Parameter.class); + AnnotationSupport.findAnnotatedFields( + context.getRequiredTestClass(), + Parameter.class, + field -> true, + HierarchyTraversalMode.BOTTOM_UP); // Use constructor parameter style if (parameterFields.isEmpty()) { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index 0b0c55f51c32..52826305b4c6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -46,12 +46,21 @@ public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Parameter(index = 0) - private String branch; + @Parameter(index = 4) + protected String branch; - @Parameters(name = "branch = {0}") + @Parameters( + name = + "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}, Branch={4}") public static Object[][] parameters() { - return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + return new Object[][] { + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "main" + }, + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "testBranch" + } + }; } @BeforeEach diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java index e1162c3225b1..2dc5bc5c658e 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -33,10 +33,12 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the FlinkSource */ public class TestFlinkSourceSql extends TestSqlBase { + @BeforeEach @Override public void before() throws IOException { SqlHelpers.sql( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 548940a842ce..66bdeee1d407 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the IcebergSource (FLIP-27) */ @@ -49,6 +50,7 @@ public class TestIcebergSourceSql extends TestSqlBase { required(1, "t1", Types.TimestampType.withoutZone()), required(2, "t2", Types.LongType.get())); + @BeforeEach @Override public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index a2e9be9303c3..56cba8f460e2 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -46,12 +46,21 @@ public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Parameter(index = 0) - private String branch; + @Parameter(index = 4) + protected String branch; - @Parameters(name = "branch = {0}") + @Parameters( + name = + "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}, Branch={4}") public static Object[][] parameters() { - return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + return new Object[][] { + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "main" + }, + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "testBranch" + } + }; } @BeforeEach diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java index 93da40ab5c9a..4896f7f48c17 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java @@ -23,9 +23,7 @@ import java.io.IOException; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; @@ -40,15 +38,8 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Branch { - @Parameter(index = 0) - private String branch; - - @Parameters(name = "branch = {0}") - public static Object[][] parameters() { - return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; - } - @BeforeEach + @Override public void before() throws IOException { table = CATALOG_EXTENSION diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java index e1162c3225b1..2dc5bc5c658e 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -33,10 +33,12 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the FlinkSource */ public class TestFlinkSourceSql extends TestSqlBase { + @BeforeEach @Override public void before() throws IOException { SqlHelpers.sql( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 548940a842ce..66bdeee1d407 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the IcebergSource (FLIP-27) */ @@ -49,6 +50,7 @@ public class TestIcebergSourceSql extends TestSqlBase { required(1, "t1", Types.TimestampType.withoutZone()), required(2, "t2", Types.LongType.get())); + @BeforeEach @Override public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index a2e9be9303c3..56cba8f460e2 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -46,12 +46,21 @@ public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Parameter(index = 0) - private String branch; + @Parameter(index = 4) + protected String branch; - @Parameters(name = "branch = {0}") + @Parameters( + name = + "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}, Branch={4}") public static Object[][] parameters() { - return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + return new Object[][] { + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "main" + }, + new Object[] { + FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, "testBranch" + } + }; } @BeforeEach diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java index 93da40ab5c9a..4896f7f48c17 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java @@ -23,9 +23,7 @@ import java.io.IOException; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; @@ -40,15 +38,8 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Branch { - @Parameter(index = 0) - private String branch; - - @Parameters(name = "branch = {0}") - public static Object[][] parameters() { - return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; - } - @BeforeEach + @Override public void before() throws IOException { table = CATALOG_EXTENSION diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java index e1162c3225b1..2dc5bc5c658e 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -33,10 +33,12 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the FlinkSource */ public class TestFlinkSourceSql extends TestSqlBase { + @BeforeEach @Override public void before() throws IOException { SqlHelpers.sql( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 548940a842ce..66bdeee1d407 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; /** Use the IcebergSource (FLIP-27) */ @@ -49,6 +50,7 @@ public class TestIcebergSourceSql extends TestSqlBase { required(1, "t1", Types.TimestampType.withoutZone()), required(2, "t2", Types.LongType.get())); + @BeforeEach @Override public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index d49c082b09d6..34fd2ad8ba31 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -63,7 +63,7 @@ jakarta-servlet-api = "6.1.0" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "11.0.24" -junit = "5.10.1" +junit = "5.11.1" junit-platform = "1.11.2" kafka = "3.8.0" kryo-shaded = "4.0.3" diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java b/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java index a5f108969249..2cb41f11295c 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java @@ -49,21 +49,18 @@ public class TestInputFormatReaderDeletes extends DeleteReadTests { private final HadoopTables tables = new HadoopTables(conf); private TestHelper helper; - // parametrized variables - @Parameter private String inputFormat; - @Parameter(index = 1) - private FileFormat fileFormat; + private String inputFormat; - @Parameters(name = "inputFormat = {0}, fileFormat = {1}") + @Parameters(name = "fileFormat = {0}, inputFormat = {1}") public static Object[][] parameters() { return new Object[][] { - {"IcebergInputFormat", FileFormat.PARQUET}, - {"IcebergInputFormat", FileFormat.AVRO}, - {"IcebergInputFormat", FileFormat.ORC}, - {"MapredIcebergInputFormat", FileFormat.PARQUET}, - {"MapredIcebergInputFormat", FileFormat.AVRO}, - {"MapredIcebergInputFormat", FileFormat.ORC}, + {FileFormat.PARQUET, "IcebergInputFormat"}, + {FileFormat.AVRO, "IcebergInputFormat"}, + {FileFormat.ORC, "IcebergInputFormat"}, + {FileFormat.PARQUET, "MapredIcebergInputFormat"}, + {FileFormat.AVRO, "MapredIcebergInputFormat"}, + {FileFormat.ORC, "MapredIcebergInputFormat"}, }; } @@ -78,9 +75,9 @@ public void writeTestDataFile() throws IOException { protected Table createTable(String name, Schema schema, PartitionSpec spec) throws IOException { Table table; - File location = temp.resolve(inputFormat).resolve(fileFormat.name()).toFile(); + File location = temp.resolve(inputFormat).resolve(format.name()).toFile(); assertThat(location.mkdirs()).isTrue(); - helper = new TestHelper(conf, tables, location.toString(), schema, spec, fileFormat, temp); + helper = new TestHelper(conf, tables, location.toString(), schema, spec, format, temp); table = helper.createTable(); TableOperations ops = ((BaseTable) table).operations(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 76a4143fcb2b..bde87778ad62 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -37,6 +37,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Parameter; @@ -98,18 +99,16 @@ public class TestSparkReaderDeletes extends DeleteReadTests { protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - @Parameter private String format; - @Parameter(index = 1) private boolean vectorized; @Parameters(name = "format = {0}, vectorized = {1}") public static Object[][] parameters() { return new Object[][] { - new Object[] {"parquet", false}, - new Object[] {"parquet", true}, - new Object[] {"orc", false}, - new Object[] {"avro", false} + new Object[] {FileFormat.PARQUET, false}, + new Object[] {FileFormat.PARQUET, true}, + new Object[] {FileFormat.ORC, false}, + new Object[] {FileFormat.AVRO, false} }; } @@ -163,14 +162,14 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { TableOperations ops = ((BaseTable) table).operations(); TableMetadata meta = ops.current(); ops.commit(meta, meta.upgradeToFormatVersion(2)); - table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit(); - if (format.equals("parquet") || format.equals("orc")) { + table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.name()).commit(); + if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { String vectorizationEnabled = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_VECTORIZATION_ENABLED : TableProperties.ORC_VECTORIZATION_ENABLED; String batchSize = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_BATCH_SIZE : TableProperties.ORC_BATCH_SIZE; table.updateProperties().set(vectorizationEnabled, String.valueOf(vectorized)).commit(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 4643836542aa..29c2d4b39a1e 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -37,6 +37,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Parameter; @@ -99,8 +100,6 @@ public class TestSparkReaderDeletes extends DeleteReadTests { protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - @Parameter private String format; - @Parameter(index = 1) private boolean vectorized; @@ -110,10 +109,10 @@ public class TestSparkReaderDeletes extends DeleteReadTests { @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") public static Object[][] parameters() { return new Object[][] { - new Object[] {"parquet", false, PlanningMode.DISTRIBUTED}, - new Object[] {"parquet", true, PlanningMode.LOCAL}, - new Object[] {"orc", false, PlanningMode.DISTRIBUTED}, - new Object[] {"avro", false, PlanningMode.LOCAL} + new Object[] {FileFormat.PARQUET, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, true, PlanningMode.LOCAL}, + new Object[] {FileFormat.ORC, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.AVRO, false, PlanningMode.LOCAL} }; } @@ -169,17 +168,17 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { ops.commit(meta, meta.upgradeToFormatVersion(2)); table .updateProperties() - .set(TableProperties.DEFAULT_FILE_FORMAT, format) + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) .set(TableProperties.DATA_PLANNING_MODE, planningMode.modeName()) .set(TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()) .commit(); - if (format.equals("parquet") || format.equals("orc")) { + if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { String vectorizationEnabled = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_VECTORIZATION_ENABLED : TableProperties.ORC_VECTORIZATION_ENABLED; String batchSize = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_BATCH_SIZE : TableProperties.ORC_BATCH_SIZE; table.updateProperties().set(vectorizationEnabled, String.valueOf(vectorized)).commit(); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java index 005d5e29d5ff..578845e3da2b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java @@ -43,6 +43,8 @@ public static void startMetastoreAndSpark() { metastore.start(); TestBase.hiveConf = metastore.hiveConf(); + TestBase.spark.close(); + TestBase.spark = SparkSession.builder() .master("local[2]") diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 4643836542aa..29c2d4b39a1e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -37,6 +37,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Parameter; @@ -99,8 +100,6 @@ public class TestSparkReaderDeletes extends DeleteReadTests { protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - @Parameter private String format; - @Parameter(index = 1) private boolean vectorized; @@ -110,10 +109,10 @@ public class TestSparkReaderDeletes extends DeleteReadTests { @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") public static Object[][] parameters() { return new Object[][] { - new Object[] {"parquet", false, PlanningMode.DISTRIBUTED}, - new Object[] {"parquet", true, PlanningMode.LOCAL}, - new Object[] {"orc", false, PlanningMode.DISTRIBUTED}, - new Object[] {"avro", false, PlanningMode.LOCAL} + new Object[] {FileFormat.PARQUET, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, true, PlanningMode.LOCAL}, + new Object[] {FileFormat.ORC, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.AVRO, false, PlanningMode.LOCAL} }; } @@ -169,17 +168,17 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { ops.commit(meta, meta.upgradeToFormatVersion(2)); table .updateProperties() - .set(TableProperties.DEFAULT_FILE_FORMAT, format) + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) .set(TableProperties.DATA_PLANNING_MODE, planningMode.modeName()) .set(TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()) .commit(); - if (format.equals("parquet") || format.equals("orc")) { + if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { String vectorizationEnabled = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_VECTORIZATION_ENABLED : TableProperties.ORC_VECTORIZATION_ENABLED; String batchSize = - format.equals("parquet") + format.equals(FileFormat.PARQUET) ? TableProperties.PARQUET_BATCH_SIZE : TableProperties.ORC_BATCH_SIZE; table.updateProperties().set(vectorizationEnabled, String.valueOf(vectorized)).commit(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java index 7e9bdeec8af0..6e09252704a1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java @@ -54,6 +54,8 @@ public static void startMetastoreAndSpark() { metastore.start(); TestBase.hiveConf = metastore.hiveConf(); + TestBase.spark.close(); + TestBase.spark = SparkSession.builder() .master("local[2]") diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java index a0c664b03b83..3df5e9cdf5da 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java @@ -22,6 +22,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; public class TestUnpartitionedWritesToBranch extends UnpartitionedWritesTestBase { @@ -29,6 +30,7 @@ public class TestUnpartitionedWritesToBranch extends UnpartitionedWritesTestBase private static final String BRANCH = "test"; @Override + @BeforeEach public void createTables() { super.createTables(); Table table = validationCatalog.loadTable(tableIdent); From 11a8a78b951d6f83fbeff703ac4e1a4b7d3f3597 Mon Sep 17 00:00:00 2001 From: Marc Cenac <547446+mrcnc@users.noreply.github.com> Date: Wed, 16 Oct 2024 10:39:50 -0500 Subject: [PATCH 005/313] Core, Azure: Support wasb[s] paths in ADLSFileIO (#11294) --- .../iceberg/azure/adlsv2/ADLSFileIO.java | 2 +- .../iceberg/azure/adlsv2/ADLSLocation.java | 61 ++++++---- .../azure/adlsv2/ADLSLocationTest.java | 105 +++++++++++++----- .../apache/iceberg/io/ResolvingFileIO.java | 4 +- 4 files changed, 116 insertions(+), 56 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java index 0bfce9d6055b..555b395e0d0e 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java @@ -111,7 +111,7 @@ DataLakeFileSystemClient client(ADLSLocation location) { new DataLakeFileSystemClientBuilder().httpClient(HTTP); location.container().ifPresent(clientBuilder::fileSystemName); - azureProperties.applyClientConfiguration(location.storageAccount(), clientBuilder); + azureProperties.applyClientConfiguration(location.storageEndpoint(), clientBuilder); return clientBuilder.buildClient(); } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java index e73093512b82..e024a5149343 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java @@ -18,26 +18,35 @@ */ package org.apache.iceberg.azure.adlsv2; +import java.net.URI; +import java.net.URISyntaxException; import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** - * This class represents a fully qualified location in Azure expressed as a URI. + * This class represents a fully qualified location in Azure Data Lake Storage, expressed as a URI. * *

Locations follow the conventions used by Hadoop's Azure support, i.e. * - *

{@code abfs[s]://[@]/}
+ *
{@code abfs[s]://[@].dfs.core.windows.net/}
* - *

See Hadoop Azure - * Support + * or + * + *

{@code wasb[s]://@.blob.core.windows.net/}
+ * + * For compatibility, paths using the wasb scheme are also accepted but will be processed via the + * Azure Data Lake Storage Gen2 APIs and not the Blob Storage APIs. + * + *

See Hadoop + * Azure Support */ class ADLSLocation { - private static final Pattern URI_PATTERN = Pattern.compile("^abfss?://([^/?#]+)(.*)?$"); + private static final Pattern URI_PATTERN = Pattern.compile("^(abfss?|wasbs?)://[^/?#]+.*$"); - private final String storageAccount; + private final String storageEndpoint; private final String container; private final String path; @@ -50,27 +59,23 @@ class ADLSLocation { Preconditions.checkArgument(location != null, "Invalid location: null"); Matcher matcher = URI_PATTERN.matcher(location); - - ValidationException.check(matcher.matches(), "Invalid ADLS URI: %s", location); - - String authority = matcher.group(1); - String[] parts = authority.split("@", -1); - if (parts.length > 1) { - this.container = parts[0]; - this.storageAccount = parts[1]; - } else { - this.container = null; - this.storageAccount = authority; + if (!matcher.matches()) { + throw new IllegalArgumentException(String.format("Invalid ADLS URI: %s", location)); } - String uriPath = matcher.group(2); - uriPath = uriPath == null ? "" : uriPath.startsWith("/") ? uriPath.substring(1) : uriPath; - this.path = uriPath.split("\\?", -1)[0].split("#", -1)[0]; + try { + URI uri = new URI(location); + this.container = uri.getUserInfo(); + this.storageEndpoint = uri.getHost(); + this.path = stripLeadingSlash(uri.getRawPath()); + } catch (URISyntaxException e) { + throw new IllegalArgumentException(String.format("Invalid ADLS URI: %s", location), e); + } } - /** Returns Azure storage account. */ - public String storageAccount() { - return storageAccount; + /** Returns Azure storage service endpoint. */ + public String storageEndpoint() { + return storageEndpoint; } /** Returns Azure container name. */ @@ -82,4 +87,12 @@ public Optional container() { public String path() { return path; } + + private static String stripLeadingSlash(String path) { + if (path.startsWith("/")) { + return path.substring(1); + } else { + return path; + } + } } diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java index 867b54b4c7e3..6edede187153 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java @@ -21,7 +21,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import org.apache.iceberg.exceptions.ValidationException; +import java.net.URI; +import java.net.URISyntaxException; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -33,17 +34,33 @@ public void testLocationParsing(String scheme) { String p1 = scheme + "://container@account.dfs.core.windows.net/path/to/file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + assertThat(location.storageEndpoint()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path/to/file"); } - @Test - public void testEncodedString() { - String p1 = "abfs://container@account.dfs.core.windows.net/path%20to%20file"; + @ParameterizedTest + @ValueSource(strings = {"wasb", "wasbs"}) + public void testWasbLocationParsing(String scheme) { + String p1 = scheme + "://container@account.blob.core.windows.net/path/to/file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + assertThat(location.storageEndpoint()).isEqualTo("account.blob.core.windows.net"); + assertThat(location.container().get()).isEqualTo("container"); + assertThat(location.path()).isEqualTo("path/to/file"); + } + + @ParameterizedTest + @ValueSource( + strings = { + "abfs://container@account.dfs.core.windows.net/path%20to%20file", + "wasb://container@account.blob.core.windows.net/path%20to%20file" + }) + public void testEncodedString(String path) throws URISyntaxException { + ADLSLocation location = new ADLSLocation(path); + String expectedEndpoint = new URI(path).getHost(); + + assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path%20to%20file"); } @@ -51,53 +68,81 @@ public void testEncodedString() { @Test public void testMissingScheme() { assertThatThrownBy(() -> new ADLSLocation("/path/to/file")) - .isInstanceOf(ValidationException.class) + .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid ADLS URI: /path/to/file"); } @Test public void testInvalidScheme() { assertThatThrownBy(() -> new ADLSLocation("s3://bucket/path/to/file")) - .isInstanceOf(ValidationException.class) + .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid ADLS URI: s3://bucket/path/to/file"); } @Test - public void testNoContainer() { - String p1 = "abfs://account.dfs.core.windows.net/path/to/file"; - ADLSLocation location = new ADLSLocation(p1); + public void testInvalidURI() { + String invalidUri = "abfs://container@account.dfs.core.windows.net/#invalidPath#"; + assertThatThrownBy(() -> new ADLSLocation(invalidUri)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(String.format("Invalid ADLS URI: %s", invalidUri)); + } - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + @ParameterizedTest + @ValueSource( + strings = { + "abfs://account.dfs.core.windows.net/path/to/file", + "wasb://account.blob.core.windows.net/path/to/file" + }) + public void testNoContainer(String path) throws URISyntaxException { + ADLSLocation location = new ADLSLocation(path); + String expectedEndpoint = new URI(path).getHost(); + + assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); assertThat(location.container().isPresent()).isFalse(); assertThat(location.path()).isEqualTo("path/to/file"); } - @Test - public void testNoPath() { - String p1 = "abfs://container@account.dfs.core.windows.net"; - ADLSLocation location = new ADLSLocation(p1); - - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + @ParameterizedTest + @ValueSource( + strings = { + "abfs://container@account.dfs.core.windows.net", + "wasb://container@account.blob.core.windows.net" + }) + public void testNoPath(String path) throws URISyntaxException { + ADLSLocation location = new ADLSLocation(path); + String expectedEndpoint = new URI(path).getHost(); + + assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo(""); } - @Test - public void testQueryAndFragment() { - String p1 = "abfs://container@account.dfs.core.windows.net/path/to/file?query=foo#123"; - ADLSLocation location = new ADLSLocation(p1); - - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + @ParameterizedTest + @ValueSource( + strings = { + "abfs://container@account.dfs.core.windows.net/path/to/file?query=foo#123", + "wasb://container@account.blob.core.windows.net/path/to/file?query=foo#123" + }) + public void testQueryAndFragment(String path) throws URISyntaxException { + ADLSLocation location = new ADLSLocation(path); + String expectedEndpoint = new URI(path).getHost(); + + assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path/to/file"); } - @Test - public void testQueryAndFragmentNoPath() { - String p1 = "abfs://container@account.dfs.core.windows.net?query=foo#123"; - ADLSLocation location = new ADLSLocation(p1); - - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + @ParameterizedTest + @ValueSource( + strings = { + "abfs://container@account.dfs.core.windows.net?query=foo#123", + "wasb://container@account.blob.core.windows.net?query=foo#123" + }) + public void testQueryAndFragmentNoPath(String path) throws URISyntaxException { + ADLSLocation location = new ADLSLocation(path); + String expectedEndpoint = new URI(path).getHost(); + + assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo(""); } diff --git a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java index a858045aab8b..a8adf979f85a 100644 --- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java +++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java @@ -62,7 +62,9 @@ public class ResolvingFileIO implements HadoopConfigurable, DelegateFileIO { "s3n", S3_FILE_IO_IMPL, "gs", GCS_FILE_IO_IMPL, "abfs", ADLS_FILE_IO_IMPL, - "abfss", ADLS_FILE_IO_IMPL); + "abfss", ADLS_FILE_IO_IMPL, + "wasb", ADLS_FILE_IO_IMPL, + "wasbs", ADLS_FILE_IO_IMPL); private final Map ioInstances = Maps.newConcurrentMap(); private final AtomicBoolean isClosed = new AtomicBoolean(false); From 22a6b19c2e226eacc0aa78c1f2ffbdbb168b13be Mon Sep 17 00:00:00 2001 From: SeungwanJo Date: Thu, 17 Oct 2024 03:04:08 +0900 Subject: [PATCH 006/313] Make connect compatable with kafka plugin.discovery (#10536) --- .../org.apache.kafka.connect.sink.SinkConnector | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 kafka-connect/kafka-connect/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector diff --git a/kafka-connect/kafka-connect/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector b/kafka-connect/kafka-connect/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector new file mode 100644 index 000000000000..0b55dc9ff070 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector @@ -0,0 +1,16 @@ +# 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. + +org.apache.iceberg.connect.IcebergSinkConnector From 17f1c4d2205b59c2bd877d4d31bbbef9e90979c5 Mon Sep 17 00:00:00 2001 From: Soumya Banerjee <48854046+jeesou@users.noreply.github.com> Date: Thu, 17 Oct 2024 03:48:26 +0530 Subject: [PATCH 007/313] Spark 3.5: Spark Scan should ignore statistics not of type Apache DataSketches (#11035) --- .../iceberg/spark/source/SparkScan.java | 34 +++-- .../iceberg/spark/source/TestSparkScan.java | 128 +++++++++++++++++- 2 files changed, 145 insertions(+), 17 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 8b88cf49c692..019f3919dc57 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -198,25 +198,31 @@ protected Statistics estimateStatistics(Snapshot snapshot) { if (!files.isEmpty()) { List metadataList = (files.get(0)).blobMetadata(); - for (BlobMetadata blobMetadata : metadataList) { - int id = blobMetadata.fields().get(0); - String colName = table.schema().findColumnName(id); + Map> groupedByField = + metadataList.stream() + .collect( + Collectors.groupingBy( + metadata -> metadata.fields().get(0), Collectors.toList())); + + for (Map.Entry> entry : groupedByField.entrySet()) { + String colName = table.schema().findColumnName(entry.getKey()); NamedReference ref = FieldReference.column(colName); - Long ndv = null; - if (blobMetadata - .type() - .equals(org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1)) { - String ndvStr = blobMetadata.properties().get(NDV_KEY); - if (!Strings.isNullOrEmpty(ndvStr)) { - ndv = Long.parseLong(ndvStr); + + for (BlobMetadata blobMetadata : entry.getValue()) { + if (blobMetadata + .type() + .equals(org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1)) { + String ndvStr = blobMetadata.properties().get(NDV_KEY); + if (!Strings.isNullOrEmpty(ndvStr)) { + ndv = Long.parseLong(ndvStr); + } else { + LOG.debug("{} is not set in BlobMetadata for column {}", NDV_KEY, colName); + } } else { - LOG.debug("ndv is not set in BlobMetadata for column {}", colName); + LOG.debug("Blob type {} is not supported yet", blobMetadata.type()); } - } else { - LOG.debug("DataSketch blob is not available for column {}", colName); } - ColumnStatistics colStats = new SparkColumnStatistics(ndv, null, null, null, null, null, null); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index 7d5475ff919e..af144fe4bf3d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -82,6 +82,8 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSparkScan extends TestBaseWithCatalog { + private static final String DUMMY_BLOB_TYPE = "sum-data-size-bytes-v1"; + @Parameter(index = 3) private String format; @@ -178,6 +180,59 @@ public void testTableWithoutColStats() throws NoSuchTableException { reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); } + @TestTemplate + public void testTableWithoutApacheDatasketchColStat() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + DUMMY_BLOB_TYPE, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("data_size", "4")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + // The expected col NDVs are nulls + withSQLConf( + reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); + } + @TestTemplate public void testTableWithOneColStats() throws NoSuchTableException { sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); @@ -232,6 +287,67 @@ public void testTableWithOneColStats() throws NoSuchTableException { withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedOneNDV)); } + @TestTemplate + public void testTableWithOneApacheDatasketchColStatAndOneDifferentColStat() + throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")), + new GenericBlobMetadata( + DUMMY_BLOB_TYPE, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("data_size", "2")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedOneNDV = Maps.newHashMap(); + expectedOneNDV.put("id", 4L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedOneNDV)); + } + @TestTemplate public void testTableWithTwoColStats() throws NoSuchTableException { sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); @@ -911,9 +1027,15 @@ private void checkColStatisticsReported( assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); Map columnStats = stats.columnStats(); - for (Map.Entry entry : expectedNDVs.entrySet()) { - assertThat(columnStats.get(FieldReference.column(entry.getKey())).distinctCount().getAsLong()) - .isEqualTo(entry.getValue()); + if (expectedNDVs.isEmpty()) { + assertThat(columnStats.values().stream().allMatch(value -> value.distinctCount().isEmpty())) + .isTrue(); + } else { + for (Map.Entry entry : expectedNDVs.entrySet()) { + assertThat( + columnStats.get(FieldReference.column(entry.getKey())).distinctCount().getAsLong()) + .isEqualTo(entry.getValue()); + } } } From 3c6c62654c50f7f5b093f82b92cfb81f6ed8ef33 Mon Sep 17 00:00:00 2001 From: RyanJClark <39035478+ryanjclark@users.noreply.github.com> Date: Wed, 16 Oct 2024 17:51:40 -0500 Subject: [PATCH 008/313] Kafka Connect: Add regex for property file match (#11303) --- .../iceberg/connect/IcebergSinkConfig.java | 10 ++++++--- .../connect/IcebergSinkConfigTest.java | 22 +++++++++++++++++++ 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java index cf34b0bcd4c8..bf5b59a0f025 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java @@ -406,6 +406,12 @@ public JsonConverter jsonConverter() { return jsonConverter; } + @VisibleForTesting + static boolean checkClassName(String className) { + return (className.matches(".*\\.ConnectDistributed.*") + || className.matches(".*\\.ConnectStandalone.*")); + } + /** * This method attempts to load the Kafka Connect worker properties, which are not exposed to * connectors. It does this by parsing the Java command used to launch the worker, extracting the @@ -422,9 +428,7 @@ private Map loadWorkerProps() { String javaCmd = System.getProperty("sun.java.command"); if (javaCmd != null && !javaCmd.isEmpty()) { List args = Splitter.on(' ').splitToList(javaCmd); - if (args.size() > 1 - && (args.get(0).endsWith(".ConnectDistributed") - || args.get(0).endsWith(".ConnectStandalone"))) { + if (args.size() > 1 && checkClassName(args.get(0))) { Properties result = new Properties(); try (InputStream in = Files.newInputStream(Paths.get(args.get(1)))) { result.load(in); diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConfigTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConfigTest.java index e2b2f96e09ba..e774b634cbff 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConfigTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConfigTest.java @@ -88,4 +88,26 @@ public void testStringToList() { @Test public void testStringWithParensToList() {} + + @Test + public void testCheckClassName() { + Boolean result = + IcebergSinkConfig.checkClassName("org.apache.kafka.connect.cli.ConnectDistributed"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("org.apache.kafka.connect.cli.ConnectStandalone"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("some.other.package.ConnectDistributed"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("some.other.package.ConnectStandalone"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("some.package.ConnectDistributedWrapper"); + assertThat(result).isTrue(); + + result = IcebergSinkConfig.checkClassName("org.apache.kafka.clients.producer.KafkaProducer"); + assertThat(result).isFalse(); + } } From 9d58865aebaea95fce43d690a0ac4711ed20fdb8 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 17 Oct 2024 15:44:53 +0900 Subject: [PATCH 009/313] OpenAPI: Remove repeated 'for' (#11338) --- open-api/rest-catalog-open-api.py | 2 +- open-api/rest-catalog-open-api.yaml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index e890604c38cd..f316dd7056a9 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -1190,7 +1190,7 @@ class LoadTableResult(BaseModel): The following configurations should be respected when working with tables stored in AWS S3 - `client.region`: region to configure client for making requests to AWS - - `s3.access-key-id`: id for for credentials that provide access to the data in S3 + - `s3.access-key-id`: id for credentials that provide access to the data in S3 - `s3.secret-access-key`: secret for credentials that provide access to data in S3 - `s3.session-token`: if present, this value should be used for as the session token - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index cf4f7d3f9458..c9fda66f155f 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -3125,7 +3125,7 @@ components: The following configurations should be respected when working with tables stored in AWS S3 - `client.region`: region to configure client for making requests to AWS - - `s3.access-key-id`: id for for credentials that provide access to the data in S3 + - `s3.access-key-id`: id for credentials that provide access to the data in S3 - `s3.secret-access-key`: secret for credentials that provide access to data in S3 - `s3.session-token`: if present, this value should be used for as the session token - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification From bbbfd1e2f14e62703d2dc5f1cb95631ab10f6d72 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Thu, 17 Oct 2024 09:48:54 -0400 Subject: [PATCH 010/313] AWS: Fix S3InputStream retry policy (#11335) --- .../apache/iceberg/aws/s3/S3InputStream.java | 26 ++++++++++++++++--- .../aws/s3/TestFlakyS3InputStream.java | 26 +++++++++++++++++++ .../iceberg/aws/s3/TestS3InputStream.java | 12 ++++++--- 3 files changed, 56 insertions(+), 8 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java index 74e602a27378..4af71932e599 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java @@ -26,6 +26,7 @@ import java.net.SocketException; import java.net.SocketTimeoutException; import java.util.Arrays; +import java.util.List; import javax.net.ssl.SSLException; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.io.FileIOMetricsContext; @@ -35,6 +36,7 @@ import org.apache.iceberg.metrics.Counter; import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.metrics.MetricsContext.Unit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -50,6 +52,9 @@ class S3InputStream extends SeekableInputStream implements RangeReadable { private static final Logger LOG = LoggerFactory.getLogger(S3InputStream.class); + private static final List> RETRYABLE_EXCEPTIONS = + ImmutableList.of(SSLException.class, SocketTimeoutException.class, SocketException.class); + private final StackTraceElement[] createStack; private final S3Client s3; private final S3URI location; @@ -66,10 +71,18 @@ class S3InputStream extends SeekableInputStream implements RangeReadable { private int skipSize = 1024 * 1024; private RetryPolicy retryPolicy = RetryPolicy.builder() - .handle( - ImmutableList.of( - SSLException.class, SocketTimeoutException.class, SocketException.class)) - .onFailure(failure -> openStream(true)) + .handle(RETRYABLE_EXCEPTIONS) + .onRetry( + e -> { + LOG.warn( + "Retrying read from S3, reopening stream (attempt {})", e.getAttemptCount()); + resetForRetry(); + }) + .onFailure( + e -> + LOG.error( + "Failed to read from S3 input stream after exhausting all retries", + e.getException())) .withMaxRetries(3) .build(); @@ -230,6 +243,11 @@ private void openStream(boolean closeQuietly) throws IOException { } } + @VisibleForTesting + void resetForRetry() throws IOException { + openStream(true); + } + private void closeStream(boolean closeQuietly) throws IOException { if (stream != null) { // if we aren't at the end of the stream, and the stream is abortable, then diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java index 08d14512cdc7..f98d1a3d4471 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.aws.s3; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; @@ -29,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Stream; import javax.net.ssl.SSLException; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -49,10 +51,29 @@ public class TestFlakyS3InputStream extends TestS3InputStream { + private AtomicInteger resetForRetryCounter; + + @BeforeEach + public void setupTest() { + resetForRetryCounter = new AtomicInteger(0); + } + + @Override + S3InputStream newInputStream(S3Client s3Client, S3URI uri) { + return new S3InputStream(s3Client, uri) { + @Override + void resetForRetry() throws IOException { + resetForRetryCounter.incrementAndGet(); + super.resetForRetry(); + } + }; + } + @ParameterizedTest @MethodSource("retryableExceptions") public void testReadWithFlakyStreamRetrySucceed(IOException exception) throws Exception { testRead(flakyStreamClient(new AtomicInteger(3), exception)); + assertThat(resetForRetryCounter.get()).isEqualTo(2); } @ParameterizedTest @@ -61,6 +82,7 @@ public void testReadWithFlakyStreamExhaustedRetries(IOException exception) { assertThatThrownBy(() -> testRead(flakyStreamClient(new AtomicInteger(5), exception))) .isInstanceOf(exception.getClass()) .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(3); } @ParameterizedTest @@ -69,12 +91,14 @@ public void testReadWithFlakyStreamNonRetryableException(IOException exception) assertThatThrownBy(() -> testRead(flakyStreamClient(new AtomicInteger(3), exception))) .isInstanceOf(exception.getClass()) .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(0); } @ParameterizedTest @MethodSource("retryableExceptions") public void testSeekWithFlakyStreamRetrySucceed(IOException exception) throws Exception { testSeek(flakyStreamClient(new AtomicInteger(3), exception)); + assertThat(resetForRetryCounter.get()).isEqualTo(2); } @ParameterizedTest @@ -83,6 +107,7 @@ public void testSeekWithFlakyStreamExhaustedRetries(IOException exception) { assertThatThrownBy(() -> testSeek(flakyStreamClient(new AtomicInteger(5), exception))) .isInstanceOf(exception.getClass()) .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(3); } @ParameterizedTest @@ -91,6 +116,7 @@ public void testSeekWithFlakyStreamNonRetryableException(IOException exception) assertThatThrownBy(() -> testSeek(flakyStreamClient(new AtomicInteger(3), exception))) .isInstanceOf(exception.getClass()) .hasMessage(exception.getMessage()); + assertThat(resetForRetryCounter.get()).isEqualTo(0); } private static Stream retryableExceptions() { diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java index 0e3f8b2136a6..f5b78eddaaad 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java @@ -57,6 +57,10 @@ public void testRead() throws Exception { testRead(s3); } + S3InputStream newInputStream(S3Client s3Client, S3URI uri) { + return new S3InputStream(s3Client, uri); + } + protected void testRead(S3Client s3Client) throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/read.dat"); int dataSize = 1024 * 1024 * 10; @@ -64,7 +68,7 @@ protected void testRead(S3Client s3Client) throws Exception { writeS3Data(uri, data); - try (SeekableInputStream in = new S3InputStream(s3Client, uri)) { + try (SeekableInputStream in = newInputStream(s3Client, uri)) { int readSize = 1024; readAndCheck(in, in.getPos(), readSize, data, false); readAndCheck(in, in.getPos(), readSize, data, true); @@ -128,7 +132,7 @@ protected void testRangeRead(S3Client s3Client) throws Exception { writeS3Data(uri, expected); - try (RangeReadable in = new S3InputStream(s3Client, uri)) { + try (RangeReadable in = newInputStream(s3Client, uri)) { // first 1k position = 0; offset = 0; @@ -160,7 +164,7 @@ private void readAndCheckRanges( @Test public void testClose() throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/closed.dat"); - SeekableInputStream closed = new S3InputStream(s3, uri); + SeekableInputStream closed = newInputStream(s3, uri); closed.close(); assertThatThrownBy(() -> closed.seek(0)) .isInstanceOf(IllegalStateException.class) @@ -178,7 +182,7 @@ protected void testSeek(S3Client s3Client) throws Exception { writeS3Data(uri, expected); - try (SeekableInputStream in = new S3InputStream(s3Client, uri)) { + try (SeekableInputStream in = newInputStream(s3Client, uri)) { in.seek(expected.length / 2); byte[] actual = new byte[expected.length / 2]; IOUtil.readFully(in, actual, 0, expected.length / 2); From 3def1f447ce2b573012802b0b5d92ee6c9e75e6a Mon Sep 17 00:00:00 2001 From: Aihua Xu Date: Thu, 17 Oct 2024 12:44:06 -0700 Subject: [PATCH 011/313] API: (Test Only) Small fix to TestSerializableTypes.java (#11342) --- .../java/org/apache/iceberg/types/TestSerializableTypes.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java index 96c330d6eb43..97bb2b1a19e4 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java @@ -112,13 +112,13 @@ public void testMaps() throws Exception { @Test public void testLists() throws Exception { - Type[] maps = + Type[] lists = new Type[] { Types.ListType.ofOptional(2, Types.DoubleType.get()), Types.ListType.ofRequired(5, Types.DoubleType.get()) }; - for (Type list : maps) { + for (Type list : lists) { Type copy = TestHelpers.roundTripSerialize(list); assertThat(copy).as("List serialization should be equal to starting type").isEqualTo(list); assertThat(list.asNestedType().asListType().elementType()) From f4ffe138968bc25b876033f36744e0573e875920 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 17 Oct 2024 14:31:02 -0700 Subject: [PATCH 012/313] Core: lazily load default Hadoop Configuration to avoid NPE with HadoopFileIO because FileIOParser doesn't serialize Hadoop configuration (#10926) Co-authored-by: Eduard Tudenhoefner --- .../apache/iceberg/hadoop/HadoopFileIO.java | 24 ++++++--- .../iceberg/hadoop/HadoopFileIOTest.java | 49 +++++++++++++++++++ 2 files changed, 66 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java index 60270749b5d7..02530cf6c9dd 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java @@ -74,7 +74,7 @@ public HadoopFileIO(SerializableSupplier hadoopConf) { } public Configuration conf() { - return hadoopConf.get(); + return getConf(); } @Override @@ -84,23 +84,23 @@ public void initialize(Map props) { @Override public InputFile newInputFile(String path) { - return HadoopInputFile.fromLocation(path, hadoopConf.get()); + return HadoopInputFile.fromLocation(path, getConf()); } @Override public InputFile newInputFile(String path, long length) { - return HadoopInputFile.fromLocation(path, length, hadoopConf.get()); + return HadoopInputFile.fromLocation(path, length, getConf()); } @Override public OutputFile newOutputFile(String path) { - return HadoopOutputFile.fromPath(new Path(path), hadoopConf.get()); + return HadoopOutputFile.fromPath(new Path(path), getConf()); } @Override public void deleteFile(String path) { Path toDelete = new Path(path); - FileSystem fs = Util.getFs(toDelete, hadoopConf.get()); + FileSystem fs = Util.getFs(toDelete, getConf()); try { fs.delete(toDelete, false /* not recursive */); } catch (IOException e) { @@ -120,6 +120,16 @@ public void setConf(Configuration conf) { @Override public Configuration getConf() { + // Create a default hadoopConf as it is required for the object to be valid. + // E.g. newInputFile would throw NPE with getConf() otherwise. + if (hadoopConf == null) { + synchronized (this) { + if (hadoopConf == null) { + this.hadoopConf = new SerializableConfiguration(new Configuration())::get; + } + } + } + return hadoopConf.get(); } @@ -132,7 +142,7 @@ public void serializeConfWith( @Override public Iterable listPrefix(String prefix) { Path prefixToList = new Path(prefix); - FileSystem fs = Util.getFs(prefixToList, hadoopConf.get()); + FileSystem fs = Util.getFs(prefixToList, getConf()); return () -> { try { @@ -154,7 +164,7 @@ public Iterable listPrefix(String prefix) { @Override public void deletePrefix(String prefix) { Path prefixToDelete = new Path(prefix); - FileSystem fs = Util.getFs(prefixToDelete, hadoopConf.get()); + FileSystem fs = Util.getFs(prefixToDelete, getConf()); try { fs.delete(prefixToDelete, true /* recursive */); diff --git a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java index 1abbd94839de..0ff1d98f27dd 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java @@ -24,6 +24,7 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.file.Files; import java.util.List; import java.util.Random; import java.util.UUID; @@ -36,6 +37,7 @@ import org.apache.iceberg.common.DynMethods; import org.apache.iceberg.io.BulkDeletionFailureException; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOParser; import org.apache.iceberg.io.ResolvingFileIO; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -176,6 +178,53 @@ public void testResolvingFileIOLoad() { assertThat(result).isInstanceOf(HadoopFileIO.class); } + @Test + public void testJsonParserWithoutHadoopConf() throws Exception { + this.hadoopFileIO = new HadoopFileIO(); + + hadoopFileIO.initialize(ImmutableMap.of("properties-bar", "2")); + assertThat(hadoopFileIO.properties().get("properties-bar")).isEqualTo("2"); + + testJsonParser(hadoopFileIO, tempDir); + } + + @Test + public void testJsonParserWithHadoopConf() throws Exception { + this.hadoopFileIO = new HadoopFileIO(); + + Configuration hadoopConf = new Configuration(); + hadoopConf.setInt("hadoop-conf-foo", 1); + hadoopFileIO.setConf(hadoopConf); + assertThat(hadoopFileIO.conf().get("hadoop-conf-foo")).isNotNull(); + + hadoopFileIO.initialize(ImmutableMap.of("properties-bar", "2")); + assertThat(hadoopFileIO.properties().get("properties-bar")).isEqualTo("2"); + + testJsonParser(hadoopFileIO, tempDir); + } + + private static void testJsonParser(HadoopFileIO hadoopFileIO, File tempDir) throws Exception { + String json = FileIOParser.toJson(hadoopFileIO); + try (FileIO deserialized = FileIOParser.fromJson(json)) { + assertThat(deserialized).isInstanceOf(HadoopFileIO.class); + HadoopFileIO deserializedHadoopFileIO = (HadoopFileIO) deserialized; + + // properties are carried over during serialization and deserialization + assertThat(deserializedHadoopFileIO.properties()).isEqualTo(hadoopFileIO.properties()); + + // FileIOParser doesn't serialize and deserialize Hadoop configuration + // so config "foo" is not restored in deserialized object. + assertThat(deserializedHadoopFileIO.conf().get("hadoop-conf-foo")).isNull(); + + // make sure deserialized io can create input file + String inputFilePath = + Files.createTempDirectory(tempDir.toPath(), "junit").toFile().getAbsolutePath() + + "/test.parquet"; + deserializedHadoopFileIO.newInputFile( + File.createTempFile("test", "parquet", tempDir).toString()); + } + } + private List createRandomFiles(Path parent, int count) { Vector paths = new Vector<>(); random From fd064386fe637e504cb71e2cfa9ebd8f28b9ac6c Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 18 Oct 2024 17:31:44 +0800 Subject: [PATCH 013/313] Revert "Core, Azure: Support wasb[s] paths in ADLSFileIO (#11294)" (#11344) This reverts commit 11a8a78b951d6f83fbeff703ac4e1a4b7d3f3597. --- .../iceberg/azure/adlsv2/ADLSFileIO.java | 2 +- .../iceberg/azure/adlsv2/ADLSLocation.java | 61 ++++------ .../azure/adlsv2/ADLSLocationTest.java | 105 +++++------------- .../apache/iceberg/io/ResolvingFileIO.java | 4 +- 4 files changed, 56 insertions(+), 116 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java index 555b395e0d0e..0bfce9d6055b 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java @@ -111,7 +111,7 @@ DataLakeFileSystemClient client(ADLSLocation location) { new DataLakeFileSystemClientBuilder().httpClient(HTTP); location.container().ifPresent(clientBuilder::fileSystemName); - azureProperties.applyClientConfiguration(location.storageEndpoint(), clientBuilder); + azureProperties.applyClientConfiguration(location.storageAccount(), clientBuilder); return clientBuilder.buildClient(); } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java index e024a5149343..e73093512b82 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java @@ -18,35 +18,26 @@ */ package org.apache.iceberg.azure.adlsv2; -import java.net.URI; -import java.net.URISyntaxException; import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** - * This class represents a fully qualified location in Azure Data Lake Storage, expressed as a URI. + * This class represents a fully qualified location in Azure expressed as a URI. * *

Locations follow the conventions used by Hadoop's Azure support, i.e. * - *

{@code abfs[s]://[@].dfs.core.windows.net/}
+ *
{@code abfs[s]://[@]/}
* - * or - * - *
{@code wasb[s]://@.blob.core.windows.net/}
- * - * For compatibility, paths using the wasb scheme are also accepted but will be processed via the - * Azure Data Lake Storage Gen2 APIs and not the Blob Storage APIs. - * - *

See Hadoop - * Azure Support + *

See Hadoop Azure + * Support */ class ADLSLocation { - private static final Pattern URI_PATTERN = Pattern.compile("^(abfss?|wasbs?)://[^/?#]+.*$"); + private static final Pattern URI_PATTERN = Pattern.compile("^abfss?://([^/?#]+)(.*)?$"); - private final String storageEndpoint; + private final String storageAccount; private final String container; private final String path; @@ -59,23 +50,27 @@ class ADLSLocation { Preconditions.checkArgument(location != null, "Invalid location: null"); Matcher matcher = URI_PATTERN.matcher(location); - if (!matcher.matches()) { - throw new IllegalArgumentException(String.format("Invalid ADLS URI: %s", location)); - } - try { - URI uri = new URI(location); - this.container = uri.getUserInfo(); - this.storageEndpoint = uri.getHost(); - this.path = stripLeadingSlash(uri.getRawPath()); - } catch (URISyntaxException e) { - throw new IllegalArgumentException(String.format("Invalid ADLS URI: %s", location), e); + ValidationException.check(matcher.matches(), "Invalid ADLS URI: %s", location); + + String authority = matcher.group(1); + String[] parts = authority.split("@", -1); + if (parts.length > 1) { + this.container = parts[0]; + this.storageAccount = parts[1]; + } else { + this.container = null; + this.storageAccount = authority; } + + String uriPath = matcher.group(2); + uriPath = uriPath == null ? "" : uriPath.startsWith("/") ? uriPath.substring(1) : uriPath; + this.path = uriPath.split("\\?", -1)[0].split("#", -1)[0]; } - /** Returns Azure storage service endpoint. */ - public String storageEndpoint() { - return storageEndpoint; + /** Returns Azure storage account. */ + public String storageAccount() { + return storageAccount; } /** Returns Azure container name. */ @@ -87,12 +82,4 @@ public Optional container() { public String path() { return path; } - - private static String stripLeadingSlash(String path) { - if (path.startsWith("/")) { - return path.substring(1); - } else { - return path; - } - } } diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java index 6edede187153..867b54b4c7e3 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java @@ -21,8 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.net.URI; -import java.net.URISyntaxException; +import org.apache.iceberg.exceptions.ValidationException; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -34,33 +33,17 @@ public void testLocationParsing(String scheme) { String p1 = scheme + "://container@account.dfs.core.windows.net/path/to/file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageEndpoint()).isEqualTo("account.dfs.core.windows.net"); + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path/to/file"); } - @ParameterizedTest - @ValueSource(strings = {"wasb", "wasbs"}) - public void testWasbLocationParsing(String scheme) { - String p1 = scheme + "://container@account.blob.core.windows.net/path/to/file"; + @Test + public void testEncodedString() { + String p1 = "abfs://container@account.dfs.core.windows.net/path%20to%20file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageEndpoint()).isEqualTo("account.blob.core.windows.net"); - assertThat(location.container().get()).isEqualTo("container"); - assertThat(location.path()).isEqualTo("path/to/file"); - } - - @ParameterizedTest - @ValueSource( - strings = { - "abfs://container@account.dfs.core.windows.net/path%20to%20file", - "wasb://container@account.blob.core.windows.net/path%20to%20file" - }) - public void testEncodedString(String path) throws URISyntaxException { - ADLSLocation location = new ADLSLocation(path); - String expectedEndpoint = new URI(path).getHost(); - - assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path%20to%20file"); } @@ -68,81 +51,53 @@ public void testEncodedString(String path) throws URISyntaxException { @Test public void testMissingScheme() { assertThatThrownBy(() -> new ADLSLocation("/path/to/file")) - .isInstanceOf(IllegalArgumentException.class) + .isInstanceOf(ValidationException.class) .hasMessage("Invalid ADLS URI: /path/to/file"); } @Test public void testInvalidScheme() { assertThatThrownBy(() -> new ADLSLocation("s3://bucket/path/to/file")) - .isInstanceOf(IllegalArgumentException.class) + .isInstanceOf(ValidationException.class) .hasMessage("Invalid ADLS URI: s3://bucket/path/to/file"); } @Test - public void testInvalidURI() { - String invalidUri = "abfs://container@account.dfs.core.windows.net/#invalidPath#"; - assertThatThrownBy(() -> new ADLSLocation(invalidUri)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage(String.format("Invalid ADLS URI: %s", invalidUri)); - } + public void testNoContainer() { + String p1 = "abfs://account.dfs.core.windows.net/path/to/file"; + ADLSLocation location = new ADLSLocation(p1); - @ParameterizedTest - @ValueSource( - strings = { - "abfs://account.dfs.core.windows.net/path/to/file", - "wasb://account.blob.core.windows.net/path/to/file" - }) - public void testNoContainer(String path) throws URISyntaxException { - ADLSLocation location = new ADLSLocation(path); - String expectedEndpoint = new URI(path).getHost(); - - assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().isPresent()).isFalse(); assertThat(location.path()).isEqualTo("path/to/file"); } - @ParameterizedTest - @ValueSource( - strings = { - "abfs://container@account.dfs.core.windows.net", - "wasb://container@account.blob.core.windows.net" - }) - public void testNoPath(String path) throws URISyntaxException { - ADLSLocation location = new ADLSLocation(path); - String expectedEndpoint = new URI(path).getHost(); - - assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); + @Test + public void testNoPath() { + String p1 = "abfs://container@account.dfs.core.windows.net"; + ADLSLocation location = new ADLSLocation(p1); + + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo(""); } - @ParameterizedTest - @ValueSource( - strings = { - "abfs://container@account.dfs.core.windows.net/path/to/file?query=foo#123", - "wasb://container@account.blob.core.windows.net/path/to/file?query=foo#123" - }) - public void testQueryAndFragment(String path) throws URISyntaxException { - ADLSLocation location = new ADLSLocation(path); - String expectedEndpoint = new URI(path).getHost(); - - assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); + @Test + public void testQueryAndFragment() { + String p1 = "abfs://container@account.dfs.core.windows.net/path/to/file?query=foo#123"; + ADLSLocation location = new ADLSLocation(p1); + + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path/to/file"); } - @ParameterizedTest - @ValueSource( - strings = { - "abfs://container@account.dfs.core.windows.net?query=foo#123", - "wasb://container@account.blob.core.windows.net?query=foo#123" - }) - public void testQueryAndFragmentNoPath(String path) throws URISyntaxException { - ADLSLocation location = new ADLSLocation(path); - String expectedEndpoint = new URI(path).getHost(); - - assertThat(location.storageEndpoint()).isEqualTo(expectedEndpoint); + @Test + public void testQueryAndFragmentNoPath() { + String p1 = "abfs://container@account.dfs.core.windows.net?query=foo#123"; + ADLSLocation location = new ADLSLocation(p1); + + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo(""); } diff --git a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java index a8adf979f85a..a858045aab8b 100644 --- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java +++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java @@ -62,9 +62,7 @@ public class ResolvingFileIO implements HadoopConfigurable, DelegateFileIO { "s3n", S3_FILE_IO_IMPL, "gs", GCS_FILE_IO_IMPL, "abfs", ADLS_FILE_IO_IMPL, - "abfss", ADLS_FILE_IO_IMPL, - "wasb", ADLS_FILE_IO_IMPL, - "wasbs", ADLS_FILE_IO_IMPL); + "abfss", ADLS_FILE_IO_IMPL); private final Map ioInstances = Maps.newConcurrentMap(); private final AtomicBoolean isClosed = new AtomicBoolean(false); From 2ac5c43284a65a9fb53438c310f15a4a362f925e Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Fri, 18 Oct 2024 09:06:57 -0700 Subject: [PATCH 014/313] Flink: make FLIP-27 default in SQL and mark the old FlinkSource as deprecated (#11345) --- docs/docs/flink-queries.md | 10 +++++++--- .../org/apache/iceberg/flink/source/FlinkSource.java | 9 +++++++++ .../org/apache/iceberg/flink/source/IcebergSource.java | 2 -- .../org/apache/iceberg/flink/source/FlinkSource.java | 9 +++++++++ .../org/apache/iceberg/flink/source/IcebergSource.java | 2 -- .../org/apache/iceberg/flink/FlinkConfigOptions.java | 2 +- .../org/apache/iceberg/flink/source/FlinkSource.java | 9 +++++++++ .../org/apache/iceberg/flink/source/IcebergSource.java | 2 -- 8 files changed, 35 insertions(+), 10 deletions(-) diff --git a/docs/docs/flink-queries.md b/docs/docs/flink-queries.md index 38263d47c199..53313db9cf5e 100644 --- a/docs/docs/flink-queries.md +++ b/docs/docs/flink-queries.md @@ -66,13 +66,17 @@ There are some options that could be set in Flink SQL hint options for streaming ### FLIP-27 source for SQL -Here are the SQL settings for the [FLIP-27](https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface) source. All other SQL settings and options documented above are applicable to the FLIP-27 source. +Here is the SQL setting to opt in or out of the +[FLIP-27 source](https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface). ```sql --- Opt in the FLIP-27 source. Default is false. -SET table.exec.iceberg.use-flip27-source = true; +-- Opt out the FLIP-27 source. +-- Default is false for Flink 1.19 and below, and true for Flink 1.20 and above. +SET table.exec.iceberg.use-flip27-source = false; ``` +All other SQL settings and options documented above are applicable to the FLIP-27 source. + ### Reading branches and tags with SQL Branch and tags can be read via SQL by specifying options. For more details refer to [Flink Configuration](flink-configuration.md#read-options) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index 228df74412a2..b5a1ba85a6cb 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -28,6 +28,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; @@ -46,6 +47,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; +/** + * Flink source builder for old {@link SourceFunction} implementation. + * + * @deprecated since 1.7.0, will be removed in 2.0.0. Use {@link IcebergSource} instead, which + * implement the newer FLIP-27 source interface. This class implements the old {@link + * SourceFunction} that has been marked as deprecated in Flink since Aug 2023. + */ +@Deprecated public class FlinkSource { private FlinkSource() {} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 294ca2f55bc5..6461d8207122 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -27,7 +27,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; -import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -86,7 +85,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Experimental public class IcebergSource implements Source { private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index 228df74412a2..b5a1ba85a6cb 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -28,6 +28,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; @@ -46,6 +47,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; +/** + * Flink source builder for old {@link SourceFunction} implementation. + * + * @deprecated since 1.7.0, will be removed in 2.0.0. Use {@link IcebergSource} instead, which + * implement the newer FLIP-27 source interface. This class implements the old {@link + * SourceFunction} that has been marked as deprecated in Flink since Aug 2023. + */ +@Deprecated public class FlinkSource { private FlinkSource() {} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 294ca2f55bc5..6461d8207122 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -27,7 +27,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; -import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -86,7 +85,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Experimental public class IcebergSource implements Source { private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java index 7c7afd24ed8e..099925101571 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java @@ -88,7 +88,7 @@ private FlinkConfigOptions() {} public static final ConfigOption TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE = ConfigOptions.key("table.exec.iceberg.use-flip27-source") .booleanType() - .defaultValue(false) + .defaultValue(true) .withDescription("Use the FLIP-27 based Iceberg source implementation."); public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index 228df74412a2..5dcb4b8cf988 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -28,6 +28,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; @@ -46,6 +47,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; +/** + * /** Flink source builder for old {@link SourceFunction} implementation. + * + * @deprecated since 1.7.0, will be removed in 2.0.0. Use {@link IcebergSource} instead, which + * implement the newer FLIP-27 source interface. This class implements the old {@link + * SourceFunction} that has been marked as deprecated in Flink since Aug 2023. + */ +@Deprecated public class FlinkSource { private FlinkSource() {} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index ba22c69c70bd..be29d4b52b69 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -27,7 +27,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; -import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -86,7 +85,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@Experimental public class IcebergSource implements Source { private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); From 8a931e89a8dc0731aa627c04cb3676b50a6a0ad3 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Fri, 18 Oct 2024 09:36:11 -0700 Subject: [PATCH 015/313] Flink: disable the flaky range distribution bucketing tests for now (#11347) --- .../sink/TestFlinkIcebergSinkRangeDistributionBucketing.java | 2 ++ .../sink/TestFlinkIcebergSinkRangeDistributionBucketing.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java index f4be472ec629..084c66317e2b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -58,6 +58,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.junit.Ignore; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -77,6 +78,7 @@ * */ @Timeout(value = 30) +@Ignore // https://github.com/apache/iceberg/pull/11305#issuecomment-2415207097 public class TestFlinkIcebergSinkRangeDistributionBucketing { private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = new Configuration() diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java index f4be472ec629..084c66317e2b 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -58,6 +58,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.junit.Ignore; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -77,6 +78,7 @@ * */ @Timeout(value = 30) +@Ignore // https://github.com/apache/iceberg/pull/11305#issuecomment-2415207097 public class TestFlinkIcebergSinkRangeDistributionBucketing { private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = new Configuration() From 44233fa5307cad4a10dfb33b67bae31da48c6798 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 18 Oct 2024 19:24:13 +0200 Subject: [PATCH 016/313] OpenAPI: Standardize credentials in loadTable/loadView responses (#10722) --- open-api/rest-catalog-open-api.py | 24 ++++++++++++++++++++++ open-api/rest-catalog-open-api.yaml | 32 +++++++++++++++++++++++++++++ 2 files changed, 56 insertions(+) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index f316dd7056a9..a16bf32155d3 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -467,6 +467,14 @@ class AssertViewUUID(BaseModel): uuid: str +class StorageCredential(BaseModel): + prefix: str = Field( + ..., + description='Indicates a storage location prefix where the credential is relevant. Clients should choose the most specific prefix (by selecting the longest prefix) if several credentials of the same type are available.', + ) + config: Dict[str, str] + + class PlanStatus(BaseModel): __root__: Literal['completed', 'submitted', 'cancelled', 'failed'] = Field( ..., description='Status of a server-side planning operation' @@ -1195,6 +1203,11 @@ class LoadTableResult(BaseModel): - `s3.session-token`: if present, this value should be used for as the session token - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification + ## Storage Credentials + + Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. + Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. + """ metadata_location: Optional[str] = Field( @@ -1204,6 +1217,9 @@ class LoadTableResult(BaseModel): ) metadata: TableMetadata config: Optional[Dict[str, str]] = None + storage_credentials: Optional[List[StorageCredential]] = Field( + None, alias='storage-credentials' + ) class ScanTasks(BaseModel): @@ -1311,11 +1327,19 @@ class LoadViewResult(BaseModel): - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled + ## Storage Credentials + + Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. + Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. + """ metadata_location: str = Field(..., alias='metadata-location') metadata: ViewMetadata config: Optional[Dict[str, str]] = None + storage_credentials: Optional[List[StorageCredential]] = Field( + None, alias='storage-credentials' + ) class ReportMetricsRequest(BaseModel): diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index c9fda66f155f..212a17c1295a 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -3103,6 +3103,21 @@ components: uuid: type: string + StorageCredential: + type: object + required: + - prefix + - config + properties: + prefix: + type: string + description: Indicates a storage location prefix where the credential is relevant. Clients should choose the most + specific prefix (by selecting the longest prefix) if several credentials of the same type are available. + config: + type: object + additionalProperties: + type: string + LoadTableResult: description: | Result used when a table is successfully loaded. @@ -3129,6 +3144,11 @@ components: - `s3.secret-access-key`: secret for credentials that provide access to data in S3 - `s3.session-token`: if present, this value should be used for as the session token - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification + + ## Storage Credentials + + Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. + Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. type: object required: - metadata @@ -3142,6 +3162,10 @@ components: type: object additionalProperties: type: string + storage-credentials: + type: array + items: + $ref: '#/components/schemas/StorageCredential' ScanTasks: type: object @@ -3395,6 +3419,10 @@ components: - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled + ## Storage Credentials + + Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. + Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. type: object required: - metadata-location @@ -3408,6 +3436,10 @@ components: type: object additionalProperties: type: string + storage-credentials: + type: array + items: + $ref: '#/components/schemas/StorageCredential' TokenType: type: string From 8dc9eacd4ea0227683710cdeb5ff6fcd4acd93fb Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 18 Oct 2024 19:24:36 +0200 Subject: [PATCH 017/313] Core: Add credentials to loadTable / loadView responses (#11173) --- .../iceberg/rest/credentials/Credential.java | 36 +++++ .../rest/credentials/CredentialParser.java | 63 ++++++++ .../rest/responses/LoadTableResponse.java | 28 +++- .../responses/LoadTableResponseParser.java | 22 +++ .../rest/responses/LoadViewResponse.java | 8 + .../responses/LoadViewResponseParser.java | 22 +++ .../credentials/TestCredentialParser.java | 143 ++++++++++++++++++ .../TestLoadTableResponseParser.java | 124 +++++++++++++++ .../responses/TestLoadViewResponseParser.java | 111 ++++++++++++++ 9 files changed, 555 insertions(+), 2 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/rest/credentials/Credential.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/credentials/CredentialParser.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/credentials/TestCredentialParser.java diff --git a/core/src/main/java/org/apache/iceberg/rest/credentials/Credential.java b/core/src/main/java/org/apache/iceberg/rest/credentials/Credential.java new file mode 100644 index 000000000000..0bd6673384de --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/credentials/Credential.java @@ -0,0 +1,36 @@ +/* + * 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.rest.credentials; + +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.immutables.value.Value; + +@Value.Immutable +public interface Credential { + String prefix(); + + Map config(); + + @Value.Check + default void validate() { + Preconditions.checkArgument(!prefix().isEmpty(), "Invalid prefix: must be non-empty"); + Preconditions.checkArgument(!config().isEmpty(), "Invalid config: must be non-empty"); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/credentials/CredentialParser.java b/core/src/main/java/org/apache/iceberg/rest/credentials/CredentialParser.java new file mode 100644 index 000000000000..14314d6e4fb2 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/credentials/CredentialParser.java @@ -0,0 +1,63 @@ +/* + * 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.rest.credentials; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.JsonUtil; + +public class CredentialParser { + private static final String PREFIX = "prefix"; + private static final String CONFIG = "config"; + + private CredentialParser() {} + + public static String toJson(Credential credential) { + return toJson(credential, false); + } + + public static String toJson(Credential credential, boolean pretty) { + return JsonUtil.generate(gen -> toJson(credential, gen), pretty); + } + + public static void toJson(Credential credential, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != credential, "Invalid credential: null"); + + gen.writeStartObject(); + + gen.writeStringField(PREFIX, credential.prefix()); + JsonUtil.writeStringMap(CONFIG, credential.config(), gen); + + gen.writeEndObject(); + } + + public static Credential fromJson(String json) { + return JsonUtil.parse(json, CredentialParser::fromJson); + } + + public static Credential fromJson(JsonNode json) { + Preconditions.checkArgument(null != json, "Cannot parse credential from null object"); + String prefix = JsonUtil.getString(PREFIX, json); + Map config = JsonUtil.getStringMap(CONFIG, json); + return ImmutableCredential.builder().prefix(prefix).config(config).build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java index 519d1fc34044..977220e7d782 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponse.java @@ -18,13 +18,17 @@ */ package org.apache.iceberg.rest.responses; +import java.util.List; import java.util.Map; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; 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.Maps; import org.apache.iceberg.rest.RESTResponse; +import org.apache.iceberg.rest.credentials.Credential; /** * A REST response that is used when a table is successfully loaded. @@ -40,16 +44,21 @@ public class LoadTableResponse implements RESTResponse { private TableMetadata metadata; private Map config; private TableMetadata metadataWithLocation; + private List credentials; public LoadTableResponse() { // Required for Jackson deserialization } private LoadTableResponse( - String metadataLocation, TableMetadata metadata, Map config) { + String metadataLocation, + TableMetadata metadata, + Map config, + List credentials) { this.metadataLocation = metadataLocation; this.metadata = metadata; this.config = config; + this.credentials = credentials; } @Override @@ -74,6 +83,10 @@ public Map config() { return config != null ? config : ImmutableMap.of(); } + public List credentials() { + return credentials != null ? credentials : ImmutableList.of(); + } + @Override public String toString() { return MoreObjects.toStringHelper(this) @@ -91,6 +104,7 @@ public static class Builder { private String metadataLocation; private TableMetadata metadata; private final Map config = Maps.newHashMap(); + private final List credentials = Lists.newArrayList(); private Builder() {} @@ -110,9 +124,19 @@ public Builder addAllConfig(Map properties) { return this; } + public Builder addCredential(Credential credential) { + credentials.add(credential); + return this; + } + + public Builder addAllCredentials(List credentialsToAdd) { + credentials.addAll(credentialsToAdd); + return this; + } + public LoadTableResponse build() { Preconditions.checkNotNull(metadata, "Invalid metadata: null"); - return new LoadTableResponse(metadataLocation, metadata, config); + return new LoadTableResponse(metadataLocation, metadata, config, credentials); } } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java index 316c5160ddc5..875403d703ab 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java @@ -24,6 +24,8 @@ import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.CredentialParser; import org.apache.iceberg.util.JsonUtil; public class LoadTableResponseParser { @@ -31,6 +33,7 @@ public class LoadTableResponseParser { private static final String METADATA_LOCATION = "metadata-location"; private static final String METADATA = "metadata"; private static final String CONFIG = "config"; + private static final String STORAGE_CREDENTIALS = "storage-credentials"; private LoadTableResponseParser() {} @@ -58,6 +61,15 @@ public static void toJson(LoadTableResponse response, JsonGenerator gen) throws JsonUtil.writeStringMap(CONFIG, response.config(), gen); } + if (!response.credentials().isEmpty()) { + gen.writeArrayFieldStart(STORAGE_CREDENTIALS); + for (Credential credential : response.credentials()) { + CredentialParser.toJson(credential, gen); + } + + gen.writeEndArray(); + } + gen.writeEndObject(); } @@ -85,6 +97,16 @@ public static LoadTableResponse fromJson(JsonNode json) { builder.addAllConfig(JsonUtil.getStringMap(CONFIG, json)); } + if (json.hasNonNull(STORAGE_CREDENTIALS)) { + JsonNode credentials = JsonUtil.get(STORAGE_CREDENTIALS, json); + Preconditions.checkArgument( + credentials.isArray(), "Cannot parse credentials from non-array: %s", credentials); + + for (JsonNode credential : credentials) { + builder.addCredential(CredentialParser.fromJson(credential)); + } + } + return builder.build(); } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java index d07ba872fdaa..d7f9040e77f7 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.rest.responses; +import java.util.List; import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.rest.RESTResponse; +import org.apache.iceberg.rest.credentials.Credential; import org.apache.iceberg.view.ViewMetadata; import org.immutables.value.Value; @@ -31,6 +34,11 @@ public interface LoadViewResponse extends RESTResponse { Map config(); + @Value.Default + default List credentials() { + return ImmutableList.of(); + } + @Override default void validate() { // nothing to validate as it's not possible to create an invalid instance diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java index a8aaf17e5d76..61d8fce1dd51 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.CredentialParser; import org.apache.iceberg.util.JsonUtil; import org.apache.iceberg.view.ViewMetadata; import org.apache.iceberg.view.ViewMetadataParser; @@ -31,6 +33,7 @@ public class LoadViewResponseParser { private static final String METADATA_LOCATION = "metadata-location"; private static final String METADATA = "metadata"; private static final String CONFIG = "config"; + private static final String STORAGE_CREDENTIALS = "storage-credentials"; private LoadViewResponseParser() {} @@ -56,6 +59,15 @@ public static void toJson(LoadViewResponse response, JsonGenerator gen) throws I JsonUtil.writeStringMap(CONFIG, response.config(), gen); } + if (!response.credentials().isEmpty()) { + gen.writeArrayFieldStart(STORAGE_CREDENTIALS); + for (Credential credential : response.credentials()) { + CredentialParser.toJson(credential, gen); + } + + gen.writeEndArray(); + } + gen.writeEndObject(); } @@ -80,6 +92,16 @@ public static LoadViewResponse fromJson(JsonNode json) { builder.config(JsonUtil.getStringMap(CONFIG, json)); } + if (json.hasNonNull(STORAGE_CREDENTIALS)) { + JsonNode credentials = JsonUtil.get(STORAGE_CREDENTIALS, json); + Preconditions.checkArgument( + credentials.isArray(), "Cannot parse credentials from non-array: %s", credentials); + + for (JsonNode credential : credentials) { + builder.addCredentials(CredentialParser.fromJson(credential)); + } + } + return builder.build(); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/credentials/TestCredentialParser.java b/core/src/test/java/org/apache/iceberg/rest/credentials/TestCredentialParser.java new file mode 100644 index 000000000000..a48fd7353b98 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/credentials/TestCredentialParser.java @@ -0,0 +1,143 @@ +/* + * 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.rest.credentials; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +public class TestCredentialParser { + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> CredentialParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid credential: null"); + + assertThatThrownBy(() -> CredentialParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse credential from null object"); + } + + @Test + public void invalidOrMissingFields() { + assertThatThrownBy(() -> CredentialParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: prefix"); + + assertThatThrownBy(() -> CredentialParser.fromJson("{\"prefix\": \"y\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing map: config"); + + assertThatThrownBy( + () -> CredentialParser.fromJson("{\"prefix\": \"\", \"config\": {\"x\": \"23\"}}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid prefix: must be non-empty"); + + assertThatThrownBy(() -> CredentialParser.fromJson("{\"prefix\": \"s3\", \"config\": {}}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid config: must be non-empty"); + } + + @Test + public void s3Credential() { + Credential credential = + ImmutableCredential.builder() + .prefix("s3://custom-uri") + .config( + ImmutableMap.of( + "s3.access-key-id", + "keyId", + "s3.secret-access-key", + "accessKey", + "s3.session-token", + "sessionToken")) + .build(); + + String expectedJson = + "{\n" + + " \"prefix\" : \"s3://custom-uri\",\n" + + " \"config\" : {\n" + + " \"s3.access-key-id\" : \"keyId\",\n" + + " \"s3.secret-access-key\" : \"accessKey\",\n" + + " \"s3.session-token\" : \"sessionToken\"\n" + + " }\n" + + "}"; + + String json = CredentialParser.toJson(credential, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(CredentialParser.toJson(CredentialParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void gcsCredential() { + Credential credential = + ImmutableCredential.builder() + .prefix("gs://custom-uri") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken", "gcs.oauth2.token-expires-at", "1000")) + .build(); + + String expectedJson = + "{\n" + + " \"prefix\" : \"gs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" + + " }\n" + + "}"; + + String json = CredentialParser.toJson(credential, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(CredentialParser.toJson(CredentialParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void adlsCredential() { + Credential credential = + ImmutableCredential.builder() + .prefix("abfs://custom-uri") + .config( + ImmutableMap.of( + "adls.sas-token.account", + "sasToken", + "adls.auth.shared-key.account.key", + "accountKey")) + .build(); + + String expectedJson = + "{\n" + + " \"prefix\" : \"abfs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"adls.sas-token.account\" : \"sasToken\",\n" + + " \"adls.auth.shared-key.account.key\" : \"accountKey\"\n" + + " }\n" + + "}"; + + String json = CredentialParser.toJson(credential, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(CredentialParser.toJson(CredentialParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java index b87c66bffe94..cc6f4cfc74d7 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java @@ -27,6 +27,7 @@ import org.apache.iceberg.SortOrder; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.credentials.ImmutableCredential; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -200,4 +201,127 @@ public void roundTripSerdeWithConfig() { assertThat(LoadTableResponseParser.toJson(LoadTableResponseParser.fromJson(json), true)) .isEqualTo(expectedJson); } + + @Test + public void roundTripSerdeWithCredentials() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + TableMetadata metadata = + TableMetadata.buildFromEmpty() + .assignUUID(uuid) + .setLocation("location") + .setCurrentSchema( + new Schema(Types.NestedField.required(1, "x", Types.LongType.get())), 1) + .addPartitionSpec(PartitionSpec.unpartitioned()) + .addSortOrder(SortOrder.unsorted()) + .discardChanges() + .withMetadataLocation("metadata-location") + .build(); + + LoadTableResponse response = + LoadTableResponse.builder() + .withTableMetadata(metadata) + .addAllConfig(ImmutableMap.of("key1", "val1", "key2", "val2")) + .addCredential( + ImmutableCredential.builder() + .prefix("s3://custom-uri") + .config( + ImmutableMap.of( + "s3.access-key-id", + "keyId", + "s3.secret-access-key", + "accessKey", + "s3.session-token", + "sessionToken")) + .build()) + .addCredential( + ImmutableCredential.builder() + .prefix("gs://custom-uri") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken1", "gcs.oauth2.token-expires-at", "1000")) + .build()) + .addCredential( + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken2", "gcs.oauth2.token-expires-at", "2000")) + .build()) + .build(); + + String expectedJson = + String.format( + "{\n" + + " \"metadata-location\" : \"metadata-location\",\n" + + " \"metadata\" : {\n" + + " \"format-version\" : 2,\n" + + " \"table-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" + + " \"location\" : \"location\",\n" + + " \"last-sequence-number\" : 0,\n" + + " \"last-updated-ms\" : %s,\n" + + " \"last-column-id\" : 1,\n" + + " \"current-schema-id\" : 0,\n" + + " \"schemas\" : [ {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"x\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"long\"\n" + + " } ]\n" + + " } ],\n" + + " \"default-spec-id\" : 0,\n" + + " \"partition-specs\" : [ {\n" + + " \"spec-id\" : 0,\n" + + " \"fields\" : [ ]\n" + + " } ],\n" + + " \"last-partition-id\" : 999,\n" + + " \"default-sort-order-id\" : 0,\n" + + " \"sort-orders\" : [ {\n" + + " \"order-id\" : 0,\n" + + " \"fields\" : [ ]\n" + + " } ],\n" + + " \"properties\" : { },\n" + + " \"current-snapshot-id\" : -1,\n" + + " \"refs\" : { },\n" + + " \"snapshots\" : [ ],\n" + + " \"statistics\" : [ ],\n" + + " \"partition-statistics\" : [ ],\n" + + " \"snapshot-log\" : [ ],\n" + + " \"metadata-log\" : [ ]\n" + + " },\n" + + " \"config\" : {\n" + + " \"key1\" : \"val1\",\n" + + " \"key2\" : \"val2\"\n" + + " },\n" + + " \"storage-credentials\" : [ {\n" + + " \"prefix\" : \"s3://custom-uri\",\n" + + " \"config\" : {\n" + + " \"s3.access-key-id\" : \"keyId\",\n" + + " \"s3.secret-access-key\" : \"accessKey\",\n" + + " \"s3.session-token\" : \"sessionToken\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken1\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken2\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"2000\"\n" + + " }\n" + + " } ]\n" + + "}", + metadata.lastUpdatedMillis()); + + String json = LoadTableResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + // can't do an equality comparison because Schema doesn't implement equals/hashCode + assertThat(LoadTableResponseParser.toJson(LoadTableResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java index f3de08cd2912..086db0fec8b4 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java @@ -25,6 +25,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.credentials.ImmutableCredential; import org.apache.iceberg.types.Types; import org.apache.iceberg.view.ImmutableViewVersion; import org.apache.iceberg.view.ViewMetadata; @@ -245,4 +246,114 @@ public void roundTripSerdeWithConfig() { assertThat(LoadViewResponseParser.toJson(LoadViewResponseParser.fromJson(json), true)) .isEqualTo(expectedJson); } + + @Test + public void roundTripSerdeWithCredentials() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + ViewMetadata viewMetadata = + ViewMetadata.builder() + .assignUUID(uuid) + .setLocation("location") + .addSchema(new Schema(Types.NestedField.required(1, "x", Types.LongType.get()))) + .addVersion( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(1) + .timestampMillis(23L) + .defaultNamespace(Namespace.of("ns1")) + .build()) + .setCurrentVersionId(1) + .build(); + + LoadViewResponse response = + ImmutableLoadViewResponse.builder() + .metadata(viewMetadata) + .metadataLocation("custom-location") + .addCredentials( + ImmutableCredential.builder() + .prefix("s3://custom-uri") + .config( + ImmutableMap.of( + "s3.access-key-id", + "keyId", + "s3.secret-access-key", + "accessKey", + "s3.session-token", + "sessionToken")) + .build()) + .addCredentials( + ImmutableCredential.builder() + .prefix("gs://custom-uri") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken1", "gcs.oauth2.token-expires-at", "1000")) + .build()) + .addCredentials( + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken2", "gcs.oauth2.token-expires-at", "2000")) + .build()) + .build(); + + String expectedJson = + "{\n" + + " \"metadata-location\" : \"custom-location\",\n" + + " \"metadata\" : {\n" + + " \"view-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" + + " \"format-version\" : 1,\n" + + " \"location\" : \"location\",\n" + + " \"schemas\" : [ {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"x\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"long\"\n" + + " } ]\n" + + " } ],\n" + + " \"current-version-id\" : 1,\n" + + " \"versions\" : [ {\n" + + " \"version-id\" : 1,\n" + + " \"timestamp-ms\" : 23,\n" + + " \"schema-id\" : 0,\n" + + " \"summary\" : { },\n" + + " \"default-namespace\" : [ \"ns1\" ],\n" + + " \"representations\" : [ ]\n" + + " } ],\n" + + " \"version-log\" : [ {\n" + + " \"timestamp-ms\" : 23,\n" + + " \"version-id\" : 1\n" + + " } ]\n" + + " },\n" + + " \"storage-credentials\" : [ {\n" + + " \"prefix\" : \"s3://custom-uri\",\n" + + " \"config\" : {\n" + + " \"s3.access-key-id\" : \"keyId\",\n" + + " \"s3.secret-access-key\" : \"accessKey\",\n" + + " \"s3.session-token\" : \"sessionToken\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken1\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken2\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"2000\"\n" + + " }\n" + + " } ]\n" + + "}"; + + String json = LoadViewResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + // can't do an equality comparison because Schema doesn't implement equals/hashCode + assertThat(LoadViewResponseParser.toJson(LoadViewResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } } From d61a98dc594e86484ea0d7d4bf8214d4ac546143 Mon Sep 17 00:00:00 2001 From: Laith AlZyoud <46904854+laithalzyoud@users.noreply.github.com> Date: Fri, 18 Oct 2024 20:01:44 +0200 Subject: [PATCH 018/313] API: Add RewriteTablePath action interface (#10920) --- .../iceberg/actions/ActionsProvider.java | 6 + .../iceberg/actions/RewriteTablePath.java | 103 ++++++++++++++++++ .../iceberg/actions/BaseRewriteTablePath.java | 33 ++++++ 3 files changed, 142 insertions(+) create mode 100644 api/src/main/java/org/apache/iceberg/actions/RewriteTablePath.java create mode 100644 core/src/main/java/org/apache/iceberg/actions/BaseRewriteTablePath.java diff --git a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java index 85773febae17..bcc77b25d698 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -76,4 +76,10 @@ default ComputeTableStats computeTableStats(Table table) { throw new UnsupportedOperationException( this.getClass().getName() + " does not implement computeTableStats"); } + + /** Instantiates an action to rewrite all absolute paths in table metadata. */ + default RewriteTablePath rewriteTablePath(Table table) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement rewriteTablePath"); + } } diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteTablePath.java b/api/src/main/java/org/apache/iceberg/actions/RewriteTablePath.java new file mode 100644 index 000000000000..b7aed67396a5 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteTablePath.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +/** + * An action that rewrites the table's metadata files to a staging directory, replacing all source + * prefixes in absolute paths with a specified target prefix. There are two modes: + * + *

    + *
  • Complete copy: Rewrites all metadata files to the staging directory. + *
  • Incremental copy: Rewrites a subset of metadata files to the staging directory, + * consisting of metadata files added since a specified start version and/or until end + * version. The start/end version is identified by the name of a metadata.json file, and all + * metadata files added before/after these file are marked for rewrite. + *
+ * + * This action can be used as the starting point to fully or incrementally copy an Iceberg table + * located under the source prefix to the target prefix. + * + *

The action returns the following: + * + *

    + *
  1. The name of the latest metadata.json rewritten to staging location. After the files are + * copied, this will be the root of the copied table. + *
  2. A list of all files added to the table between startVersion and endVersion, including their + * original and target paths under the target prefix. This list covers both original and + * rewritten files, allowing for copying to the target paths to form the copied table. + *
+ */ +public interface RewriteTablePath extends Action { + + /** + * Configure a source prefix that will be replaced by the specified target prefix in all paths + * + * @param sourcePrefix the source prefix to be replaced + * @param targetPrefix the target prefix + * @return this for method chaining + */ + RewriteTablePath rewriteLocationPrefix(String sourcePrefix, String targetPrefix); + + /** + * First metadata version to rewrite, identified by name of a metadata.json file in the table's + * metadata log. It is optional, if provided then this action will only rewrite metadata files + * added after this version. + * + * @param startVersion name of a metadata.json file. For example, + * "00001-8893aa9e-f92e-4443-80e7-cfa42238a654.metadata.json". + * @return this for method chaining + */ + RewriteTablePath startVersion(String startVersion); + + /** + * Last metadata version to rewrite, identified by name of a metadata.json file in the table's + * metadata log. It is optional, if provided then this action will only rewrite metadata files + * added before this file, including the file itself. + * + * @param endVersion name of a metadata.json file. For example, + * "00001-8893aa9e-f92e-4443-80e7-cfa42238a654.metadata.json". + * @return this for method chaining + */ + RewriteTablePath endVersion(String endVersion); + + /** + * Custom staging location. It is optional. By default, staging location is a subdirectory under + * table's metadata directory. + * + * @param stagingLocation the staging location + * @return this for method chaining + */ + RewriteTablePath stagingLocation(String stagingLocation); + + /** The action result that contains a summary of the execution. */ + interface Result { + /** Staging location of rewritten files */ + String stagingLocation(); + + /** + * Path to a comma-separated list of source and target paths for all files added to the table + * between startVersion and endVersion, including original data files and metadata files + * rewritten to staging. + */ + String fileListLocation(); + + /** Name of latest metadata file version */ + String latestVersion(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteTablePath.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteTablePath.java new file mode 100644 index 000000000000..1fb343da92fe --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteTablePath.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import org.immutables.value.Value; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableRewriteTablePath", + visibilityString = "PUBLIC", + builderVisibilityString = "PUBLIC") +interface BaseRewriteTablePath extends RewriteTablePath { + + @Value.Immutable + interface Result extends RewriteTablePath.Result {} +} From ea5da1789f3eb80fc3196bcdd787a95ac0c493ba Mon Sep 17 00:00:00 2001 From: Ozan Okumusoglu Date: Fri, 18 Oct 2024 16:48:00 -0700 Subject: [PATCH 019/313] AWS: Switch to base2 entropy in ObjectStoreLocationProvider for optimized S3 performance (#11112) Co-authored-by: Drew Schleit --- .../org/apache/iceberg/LocationProviders.java | 69 ++++++++++++++++--- .../org/apache/iceberg/TableProperties.java | 5 ++ .../apache/iceberg/TestLocationProvider.java | 40 +++++++++-- docs/docs/aws.md | 26 +++++-- docs/docs/configuration.md | 1 + 5 files changed, 123 insertions(+), 18 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/LocationProviders.java b/core/src/main/java/org/apache/iceberg/LocationProviders.java index 0c4924b6d5a6..68bec2f4e4fc 100644 --- a/core/src/main/java/org/apache/iceberg/LocationProviders.java +++ b/core/src/main/java/org/apache/iceberg/LocationProviders.java @@ -27,7 +27,6 @@ import org.apache.iceberg.relocated.com.google.common.hash.HashCode; import org.apache.iceberg.relocated.com.google.common.hash.HashFunction; import org.apache.iceberg.relocated.com.google.common.hash.Hashing; -import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.apache.iceberg.util.LocationUtil; import org.apache.iceberg.util.PropertyUtil; @@ -108,10 +107,15 @@ public String newDataLocation(String filename) { static class ObjectStoreLocationProvider implements LocationProvider { private static final HashFunction HASH_FUNC = Hashing.murmur3_32_fixed(); - private static final BaseEncoding BASE64_ENCODER = BaseEncoding.base64Url().omitPadding(); - private static final ThreadLocal TEMP = ThreadLocal.withInitial(() -> new byte[4]); + // Length of entropy generated in the file location + private static final int HASH_BINARY_STRING_BITS = 20; + // Entropy generated will be divided into dirs with this lengths + private static final int ENTROPY_DIR_LENGTH = 4; + // Will create DEPTH many dirs from the entropy + private static final int ENTROPY_DIR_DEPTH = 3; private final String storageLocation; private final String context; + private final boolean includePartitionPaths; ObjectStoreLocationProvider(String tableLocation, Map properties) { this.storageLocation = @@ -123,6 +127,11 @@ static class ObjectStoreLocationProvider implements LocationProvider { } else { this.context = pathContext(tableLocation); } + this.includePartitionPaths = + PropertyUtil.propertyAsBoolean( + properties, + TableProperties.WRITE_OBJECT_STORE_PARTITIONED_PATHS, + TableProperties.WRITE_OBJECT_STORE_PARTITIONED_PATHS_DEFAULT); } private static String dataLocation(Map properties, String tableLocation) { @@ -141,7 +150,12 @@ private static String dataLocation(Map properties, String tableL @Override public String newDataLocation(PartitionSpec spec, StructLike partitionData, String filename) { - return newDataLocation(String.format("%s/%s", spec.partitionToPath(partitionData), filename)); + if (includePartitionPaths) { + return newDataLocation( + String.format("%s/%s", spec.partitionToPath(partitionData), filename)); + } else { + return newDataLocation(filename); + } } @Override @@ -150,7 +164,13 @@ public String newDataLocation(String filename) { if (context != null) { return String.format("%s/%s/%s/%s", storageLocation, hash, context, filename); } else { - return String.format("%s/%s/%s", storageLocation, hash, filename); + // if partition paths are included, add last part of entropy as dir before partition names + if (includePartitionPaths) { + return String.format("%s/%s/%s", storageLocation, hash, filename); + } else { + // if partition paths are not included, append last part of entropy with `-` to file name + return String.format("%s/%s-%s", storageLocation, hash, filename); + } } } @@ -172,10 +192,41 @@ private static String pathContext(String tableLocation) { } private String computeHash(String fileName) { - byte[] bytes = TEMP.get(); - HashCode hash = HASH_FUNC.hashString(fileName, StandardCharsets.UTF_8); - hash.writeBytesTo(bytes, 0, 4); - return BASE64_ENCODER.encode(bytes); + HashCode hashCode = HASH_FUNC.hashString(fileName, StandardCharsets.UTF_8); + + // {@link Integer#toBinaryString} excludes leading zeros, which we want to preserve. + // force the first bit to be set to get around that. + String hashAsBinaryString = Integer.toBinaryString(hashCode.asInt() | Integer.MIN_VALUE); + // Limit hash length to HASH_BINARY_STRING_BITS + String hash = + hashAsBinaryString.substring(hashAsBinaryString.length() - HASH_BINARY_STRING_BITS); + return dirsFromHash(hash); + } + + /** + * Divides hash into directories for optimized orphan removal operation using ENTROPY_DIR_DEPTH + * and ENTROPY_DIR_LENGTH + * + * @param hash 10011001100110011001 + * @return 1001/1001/1001/10011001 with depth 3 and length 4 + */ + private String dirsFromHash(String hash) { + StringBuilder hashWithDirs = new StringBuilder(); + + for (int i = 0; i < ENTROPY_DIR_DEPTH * ENTROPY_DIR_LENGTH; i += ENTROPY_DIR_LENGTH) { + if (i > 0) { + hashWithDirs.append("/"); + } + hashWithDirs.append(hash, i, Math.min(i + ENTROPY_DIR_LENGTH, hash.length())); + } + + if (hash.length() > ENTROPY_DIR_DEPTH * ENTROPY_DIR_LENGTH) { + hashWithDirs + .append("/") + .append(hash, ENTROPY_DIR_DEPTH * ENTROPY_DIR_LENGTH, hash.length()); + } + + return hashWithDirs.toString(); } } } diff --git a/core/src/main/java/org/apache/iceberg/TableProperties.java b/core/src/main/java/org/apache/iceberg/TableProperties.java index 1652e9312f50..c137bcd3a2c3 100644 --- a/core/src/main/java/org/apache/iceberg/TableProperties.java +++ b/core/src/main/java/org/apache/iceberg/TableProperties.java @@ -244,6 +244,11 @@ private TableProperties() {} public static final String OBJECT_STORE_ENABLED = "write.object-storage.enabled"; public static final boolean OBJECT_STORE_ENABLED_DEFAULT = false; + // Excludes the partition values in the path when set to true and object store is enabled + public static final String WRITE_OBJECT_STORE_PARTITIONED_PATHS = + "write.object-storage.partitioned-paths"; + public static final boolean WRITE_OBJECT_STORE_PARTITIONED_PATHS_DEFAULT = true; + /** * @deprecated Use {@link #WRITE_DATA_LOCATION} instead. */ diff --git a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java index 7afb69483490..7edba51c3d85 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java +++ b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java @@ -240,7 +240,6 @@ public void testObjectStorageLocationProviderPathResolution() { String dataPath = "s3://random/data/location"; table.updateProperties().set(TableProperties.WRITE_DATA_LOCATION, dataPath).commit(); - assertThat(table.locationProvider().newDataLocation("file")) .as("write data path should be used when set") .contains(dataPath); @@ -279,12 +278,12 @@ public void testObjectStorageWithinTableLocation() { String fileLocation = table.locationProvider().newDataLocation("test.parquet"); String relativeLocation = fileLocation.replaceFirst(table.location(), ""); List parts = Splitter.on("/").splitToList(relativeLocation); - - assertThat(parts).hasSize(4); + assertThat(parts).hasSize(7); assertThat(parts).first().asString().isEmpty(); assertThat(parts).element(1).asString().isEqualTo("data"); - assertThat(parts).element(2).asString().isNotEmpty(); - assertThat(parts).element(3).asString().isEqualTo("test.parquet"); + // entropy dirs in the middle + assertThat(parts).elements(2, 3, 4, 5).asString().isNotEmpty(); + assertThat(parts).element(6).asString().isEqualTo("test.parquet"); } @TestTemplate @@ -304,4 +303,35 @@ public void testEncodedFieldNameInPartitionPath() { assertThat(partitionString).isEqualTo("data%231=val%231"); } + + @TestTemplate + public void testExcludePartitionInPath() { + // Update the table to use a string field for partitioning with special characters in the name + table.updateProperties().set(TableProperties.OBJECT_STORE_ENABLED, "true").commit(); + table + .updateProperties() + .set(TableProperties.WRITE_OBJECT_STORE_PARTITIONED_PATHS, "false") + .commit(); + + // Use a partition value that has a special character + StructLike partitionData = TestHelpers.CustomRow.of(0, "val"); + String fileLocation = + table.locationProvider().newDataLocation(table.spec(), partitionData, "test.parquet"); + + // no partition values included in the path and last part of entropy is seperated with "-" + assertThat(fileLocation).endsWith("/data/0110/1010/0011/11101000-test.parquet"); + } + + @TestTemplate + public void testHashInjection() { + table.updateProperties().set(TableProperties.OBJECT_STORE_ENABLED, "true").commit(); + assertThat(table.locationProvider().newDataLocation("a")) + .endsWith("/data/0101/0110/1001/10110010/a"); + assertThat(table.locationProvider().newDataLocation("b")) + .endsWith("/data/1110/0111/1110/00000011/b"); + assertThat(table.locationProvider().newDataLocation("c")) + .endsWith("/data/0010/1101/0110/01011111/c"); + assertThat(table.locationProvider().newDataLocation("d")) + .endsWith("/data/1001/0001/0100/01110011/d"); + } } diff --git a/docs/docs/aws.md b/docs/docs/aws.md index e408cb5a2ae4..1a98a4d18e5b 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -343,7 +343,10 @@ Data stored in S3 with a traditional Hive storage layout can face S3 request thr Iceberg by default uses the Hive storage layout but can be switched to use the `ObjectStoreLocationProvider`. With `ObjectStoreLocationProvider`, a deterministic hash is generated for each stored file, with the hash appended -directly after the `write.data.path`. This ensures files written to s3 are equally distributed across multiple [prefixes](https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/) in the S3 bucket. Resulting in minimized throttling and maximized throughput for S3-related IO operations. When using `ObjectStoreLocationProvider` having a shared and short `write.data.path` across your Iceberg tables will improve performance. +directly after the `write.data.path`. This ensures files written to S3 are equally distributed across multiple +[prefixes](https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/) in the S3 bucket; +resulting in minimized throttling and maximized throughput for S3-related IO operations. When using `ObjectStoreLocationProvider` +having a shared `write.data.path` across your Iceberg tables will improve performance. For more information on how S3 scales API QPS, check out the 2018 re:Invent session on [Best Practices for Amazon S3 and Amazon S3 Glacier](https://youtu.be/rHeTn9pHNKo?t=3219). At [53:39](https://youtu.be/rHeTn9pHNKo?t=3219) it covers how S3 scales/partitions & at [54:50](https://youtu.be/rHeTn9pHNKo?t=3290) it discusses the 30-60 minute wait time before new partitions are created. @@ -357,7 +360,7 @@ CREATE TABLE my_catalog.my_ns.my_table ( USING iceberg OPTIONS ( 'write.object-storage.enabled'=true, - 'write.data.path'='s3://my-table-data-bucket') + 'write.data.path'='s3://my-table-data-bucket/my_table') PARTITIONED BY (category); ``` @@ -366,9 +369,16 @@ We can then insert a single row into this new table INSERT INTO my_catalog.my_ns.my_table VALUES (1, "Pizza", "orders"); ``` -Which will write the data to S3 with a hash (`2d3905f8`) appended directly after the `write.object-storage.path`, ensuring reads to the table are spread evenly across [S3 bucket prefixes](https://docs.aws.amazon.com/AmazonS3/latest/userguide/optimizing-performance.html), and improving performance. +Which will write the data to S3 with a 20-bit base2 hash (`01010110100110110010`) appended directly after the `write.object-storage.path`, +ensuring reads to the table are spread evenly across [S3 bucket prefixes](https://docs.aws.amazon.com/AmazonS3/latest/userguide/optimizing-performance.html), and improving performance. +Previously provided base64 hash was updated to base2 in order to provide an improved auto-scaling behavior on S3 General Purpose Buckets. + +As part of this update, we have also divided the entropy into multiple directories in order to improve the efficiency of the +orphan clean up process for Iceberg since directories are used as a mean to divide the work across workers for faster traversal. You +can see from the example below that we divide the hash to create 4-bit directories with a depth of 3 and attach the final part of the hash to +the end. ``` -s3://my-table-data-bucket/2d3905f8/my_ns.db/my_table/category=orders/00000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet +s3://my-table-data-bucket/my_ns.db/my_table/0101/0110/1001/10110010/category=orders/00000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet ``` Note, the path resolution logic for `ObjectStoreLocationProvider` is `write.data.path` then `/data`. @@ -378,6 +388,14 @@ However, for the older versions up to 0.12.0, the logic is as follows: For more details, please refer to the [LocationProvider Configuration](custom-catalog.md#custom-location-provider-implementation) section. +We have also added a new table property `write.object-storage.partitioned-paths` that if set to false(default=true), this will +omit the partition values from the file path. Iceberg does not need these values in the file path and setting this value to false +can further reduce the key size. In this case, we also append the final 8 bit of entropy directly to the file name. +Inserted key would look like the following with this config set, note that `category=orders` is removed: +``` +s3://my-table-data-bucket/my_ns.db/my_table/1101/0100/1011/00111010-00000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet +``` + ### S3 Retries Workloads which encounter S3 throttling should persistently retry, with exponential backoff, to make progress while S3 diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index 264b9edfa7cc..07a98fd94515 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -77,6 +77,7 @@ Iceberg tables support table properties to configure table behavior, like the de | write.metadata.previous-versions-max | 100 | The max number of previous version metadata files to keep before deleting after commit | | write.spark.fanout.enabled | false | Enables the fanout writer in Spark that does not require data to be clustered; uses more memory | | write.object-storage.enabled | false | Enables the object storage location provider that adds a hash component to file paths | +| write.object-storage.partitioned-paths | true | Includes the partition values in the file path | | write.data.path | table location + /data | Base location for data files | | write.metadata.path | table location + /metadata | Base location for metadata files | | write.delete.mode | copy-on-write | Mode used for delete commands: copy-on-write or merge-on-read (v2 only) | From d4f0d7e8036488924edf9acdc2aed6fc3a3415e1 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Mon, 21 Oct 2024 06:35:03 +0200 Subject: [PATCH 020/313] Flink: Add IcebergSinkBuilder interface allowed unification of most of operations on FlinkSink and IcebergSink Builders (#11305) --- .../apache/iceberg/flink/sink/FlinkSink.java | 14 +++- .../iceberg/flink/sink/IcebergSink.java | 14 +++- .../flink/sink/IcebergSinkBuilder.java | 83 +++++++++++++++++++ 3 files changed, 109 insertions(+), 2 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index c53431490984..e862e88c968c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -134,7 +134,7 @@ public static Builder forRowData(DataStream input) { return new Builder().forRowData(input); } - public static class Builder { + public static class Builder implements IcebergSinkBuilder { private Function> inputCreator = null; private TableLoader tableLoader; private Table table; @@ -179,6 +179,7 @@ private Builder forMapperOutputType( * @param newTable the loaded iceberg table instance. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder table(Table newTable) { this.table = newTable; return this; @@ -192,6 +193,7 @@ public Builder table(Table newTable) { * @param newTableLoader to load iceberg table inside tasks. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder tableLoader(TableLoader newTableLoader) { this.tableLoader = newTableLoader; return this; @@ -210,21 +212,25 @@ public Builder set(String property, String value) { * Set the write properties for Flink sink. View the supported properties in {@link * FlinkWriteOptions} */ + @Override public Builder setAll(Map properties) { writeOptions.putAll(properties); return this; } + @Override public Builder tableSchema(TableSchema newTableSchema) { this.tableSchema = newTableSchema; return this; } + @Override public Builder overwrite(boolean newOverwrite) { writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); return this; } + @Override public Builder flinkConf(ReadableConfig config) { this.readableConfig = config; return this; @@ -237,6 +243,7 @@ public Builder flinkConf(ReadableConfig config) { * @param mode to specify the write distribution mode. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder distributionMode(DistributionMode mode) { if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); @@ -306,6 +313,7 @@ public Builder rangeDistributionSortKeyBaseWeight(double weight) { * @param newWriteParallelism the number of parallel iceberg stream writer. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder writeParallelism(int newWriteParallelism) { writeOptions.put( FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); @@ -321,6 +329,7 @@ public Builder writeParallelism(int newWriteParallelism) { * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder upsert(boolean enabled) { writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); return this; @@ -332,6 +341,7 @@ public Builder upsert(boolean enabled) { * @param columns defines the iceberg table's key. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder equalityFieldColumns(List columns) { this.equalityFieldColumns = columns; return this; @@ -376,6 +386,7 @@ public Builder setSnapshotProperty(String property, String value) { return this; } + @Override public Builder toBranch(String branch) { writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); return this; @@ -436,6 +447,7 @@ private DataStreamSink chainIcebergOperators() { * * @return {@link DataStreamSink} for sink. */ + @Override public DataStreamSink append() { return chainIcebergOperators(); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java index d080169544cd..01be4a2eef71 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -255,7 +255,7 @@ public SimpleVersionedSerializer getWriteResultSerializer() { return new WriteResultSerializer(); } - public static class Builder { + public static class Builder implements IcebergSinkBuilder { private TableLoader tableLoader; private String uidSuffix = ""; private Function> inputCreator = null; @@ -311,6 +311,7 @@ private Builder forMapperOutputType( * @param newTable the loaded iceberg table instance. * @return {@link IcebergSink.Builder} to connect the iceberg table. */ + @Override public Builder table(Table newTable) { this.table = (SerializableTable) SerializableTable.copyOf(newTable); return this; @@ -325,6 +326,7 @@ public Builder table(Table newTable) { * @param newTableLoader to load iceberg table inside tasks. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder tableLoader(TableLoader newTableLoader) { this.tableLoader = newTableLoader; return this; @@ -347,21 +349,25 @@ public Builder set(String property, String value) { * Set the write properties for IcebergSink. View the supported properties in {@link * FlinkWriteOptions} */ + @Override public Builder setAll(Map properties) { writeOptions.putAll(properties); return this; } + @Override public Builder tableSchema(TableSchema newTableSchema) { this.tableSchema = newTableSchema; return this; } + @Override public Builder overwrite(boolean newOverwrite) { writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); return this; } + @Override public Builder flinkConf(ReadableConfig config) { this.readableConfig = config; return this; @@ -374,6 +380,7 @@ public Builder flinkConf(ReadableConfig config) { * @param mode to specify the write distribution mode. * @return {@link IcebergSink.Builder} to connect the iceberg table. */ + @Override public Builder distributionMode(DistributionMode mode) { Preconditions.checkArgument( !DistributionMode.RANGE.equals(mode), @@ -390,6 +397,7 @@ public Builder distributionMode(DistributionMode mode) { * @param newWriteParallelism the number of parallel iceberg stream writer. * @return {@link IcebergSink.Builder} to connect the iceberg table. */ + @Override public Builder writeParallelism(int newWriteParallelism) { writeOptions.put( FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); @@ -405,6 +413,7 @@ public Builder writeParallelism(int newWriteParallelism) { * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. * @return {@link IcebergSink.Builder} to connect the iceberg table. */ + @Override public Builder upsert(boolean enabled) { writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); return this; @@ -416,6 +425,7 @@ public Builder upsert(boolean enabled) { * @param columns defines the iceberg table's key. * @return {@link Builder} to connect the iceberg table. */ + @Override public Builder equalityFieldColumns(List columns) { this.equalityFieldColumns = columns; return this; @@ -458,6 +468,7 @@ public Builder setSnapshotProperty(String property, String value) { return this; } + @Override public Builder toBranch(String branch) { writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); return this; @@ -527,6 +538,7 @@ IcebergSink build() { * * @return {@link DataStreamSink} for sink. */ + @Override public DataStreamSink append() { IcebergSink sink = build(); String suffix = defaultSuffix(uidSuffix, table.name()); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.java new file mode 100644 index 000000000000..f232df512894 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkBuilder.java @@ -0,0 +1,83 @@ +/* + * 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.flink.sink; + +import java.util.List; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; + +/** + * This class is for internal purpose of transition between the previous implementation of Flink's + * sink ({@link FlinkSink}) and the new one implementation based on Flink SinkV2 API ({@link + * IcebergSink}). After we remove the previous implementation, all occurrences of this class would + * be replaced by direct {@link IcebergSink} usage. + */ +@Internal +interface IcebergSinkBuilder> { + + T tableSchema(TableSchema newTableSchema); + + T tableLoader(TableLoader newTableLoader); + + T equalityFieldColumns(List columns); + + T overwrite(boolean newOverwrite); + + T setAll(Map properties); + + T flinkConf(ReadableConfig config); + + T table(Table newTable); + + T writeParallelism(int newWriteParallelism); + + T distributionMode(DistributionMode mode); + + T toBranch(String branch); + + T upsert(boolean enabled); + + DataStreamSink append(); + + static IcebergSinkBuilder forRow( + DataStream input, TableSchema tableSchema, boolean useV2Sink) { + if (useV2Sink) { + return IcebergSink.forRow(input, tableSchema); + } else { + return FlinkSink.forRow(input, tableSchema); + } + } + + static IcebergSinkBuilder forRowData(DataStream input, boolean useV2Sink) { + if (useV2Sink) { + return IcebergSink.forRowData(input); + } else { + return FlinkSink.forRowData(input); + } + } +} From b8c2b20237bc9309d34dc96c473e9941d1b2ad58 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 21 Oct 2024 07:34:44 +0200 Subject: [PATCH 021/313] Build: Bump parquet from 1.13.1 to 1.14.3 (#11264) Co-authored-by: Eduard Tudenhoefner --- .../TestMetadataTableReadableMetrics.java | 22 +++++++++---------- .../TestMetadataTableReadableMetrics.java | 22 +++++++++---------- .../TestMetadataTableReadableMetrics.java | 22 +++++++++---------- gradle/libs.versions.toml | 2 +- 4 files changed, 34 insertions(+), 34 deletions(-) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 40dfda723749..3aa2a7c0667f 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -217,27 +217,27 @@ public void testPrimitiveColumns() throws Exception { Row binaryCol = Row.of( - 52L, + 55L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = Row.of(36L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(91L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(91L, 4L, 0L, 1L, 1.0D, 2.0D); Row fixedCol = Row.of( - 44L, + 47L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); + Row floatCol = Row.of(77L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(77L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(85L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(85L, 4L, 0L, null, "1", "2"); List expected = Lists.newArrayList( @@ -289,8 +289,8 @@ public void testSelectNestedValues() throws Exception { public void testNestedValues() throws Exception { createNestedTable(); - Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(50L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(57L, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); TestHelpers.assertRows( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 9cf953342a18..9cdcb72c12cf 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -217,27 +217,27 @@ public void testPrimitiveColumns() throws Exception { Row binaryCol = Row.of( - 52L, + 55L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = Row.of(36L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(91L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(91L, 4L, 0L, 1L, 1.0D, 2.0D); Row fixedCol = Row.of( - 44L, + 47L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); + Row floatCol = Row.of(77L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(77L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(85L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(85L, 4L, 0L, null, "1", "2"); List expected = Lists.newArrayList( @@ -289,8 +289,8 @@ public void testSelectNestedValues() throws Exception { public void testNestedValues() throws Exception { createNestedTable(); - Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(50L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(57L, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); TestHelpers.assertRows( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 9cf953342a18..9cdcb72c12cf 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -217,27 +217,27 @@ public void testPrimitiveColumns() throws Exception { Row binaryCol = Row.of( - 52L, + 55L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = Row.of(36L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(91L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(91L, 4L, 0L, 1L, 1.0D, 2.0D); Row fixedCol = Row.of( - 44L, + 47L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); + Row floatCol = Row.of(77L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(77L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(85L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(85L, 4L, 0L, null, "1", "2"); List expected = Lists.newArrayList( @@ -289,8 +289,8 @@ public void testSelectNestedValues() throws Exception { public void testNestedValues() throws Exception { createNestedTable(); - Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(50L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(57L, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); TestHelpers.assertRows( diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 34fd2ad8ba31..8b3a796b61b4 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -75,7 +75,7 @@ netty-buffer = "4.1.114.Final" netty-buffer-compat = "4.1.114.Final" object-client-bundle = "3.3.2" orc = "1.9.4" -parquet = "1.13.1" +parquet = "1.14.3" pig = "0.17.0" roaringbitmap = "1.3.0" s3mock-junit5 = "2.17.0" From de48a74263448a191344d411004db67900f528a8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 21 Oct 2024 09:06:59 +0200 Subject: [PATCH 022/313] Build: Bump com.palantir.baseline:gradle-baseline-java (#11362) Bumps [com.palantir.baseline:gradle-baseline-java](https://github.com/palantir/gradle-baseline) from 5.69.0 to 5.72.0. - [Release notes](https://github.com/palantir/gradle-baseline/releases) - [Changelog](https://github.com/palantir/gradle-baseline/blob/develop/.changelog.yml) - [Commits](https://github.com/palantir/gradle-baseline/compare/5.69.0...5.72.0) --- updated-dependencies: - dependency-name: com.palantir.baseline:gradle-baseline-java dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index e66ea793bbf7..9e4b62d87576 100644 --- a/build.gradle +++ b/build.gradle @@ -27,7 +27,7 @@ buildscript { } dependencies { classpath 'com.gradleup.shadow:shadow-gradle-plugin:8.3.3' - classpath 'com.palantir.baseline:gradle-baseline-java:5.69.0' + classpath 'com.palantir.baseline:gradle-baseline-java:5.72.0' classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.25.0' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.2' From 181476471a75afcee1768ddf99afd88b6e348006 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 21 Oct 2024 09:07:18 +0200 Subject: [PATCH 023/313] Build: Bump com.google.errorprone:error_prone_annotations (#11360) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.33.0 to 2.34.0. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.33.0...v2.34.0) --- updated-dependencies: - dependency-name: com.google.errorprone:error_prone_annotations dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 8b3a796b61b4..e9a438cbb231 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -38,7 +38,7 @@ datasketches = "6.1.1" delta-standalone = "3.2.1" delta-spark = "3.2.1" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.33.0" +errorprone-annotations = "2.34.0" failsafe = "3.3.2" findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} From 0640a38b6b030b0e1c3ef7bb3d92e4c64b621dc7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 21 Oct 2024 09:07:34 +0200 Subject: [PATCH 024/313] Build: Bump software.amazon.awssdk:bom from 2.28.21 to 2.28.26 (#11359) Bumps software.amazon.awssdk:bom from 2.28.21 to 2.28.26. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index e9a438cbb231..8de810eb918f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.28.21" +awssdk-bom = "2.28.26" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.2.0" caffeine = "2.9.3" From ce75f52719999d1b3aaba3ed4f10d4ca7fdb8bb8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 21 Oct 2024 12:00:36 +0200 Subject: [PATCH 025/313] Build: Bump com.google.cloud:libraries-bom from 26.48.0 to 26.49.0 (#11363) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.48.0 to 26.49.0. - [Release notes](https://github.com/googleapis/java-cloud-bom/releases) - [Changelog](https://github.com/googleapis/java-cloud-bom/blob/main/release-please-config.json) - [Commits](https://github.com/googleapis/java-cloud-bom/compare/v26.48.0...v26.49.0) --- updated-dependencies: - dependency-name: com.google.cloud:libraries-bom dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 8de810eb918f..deb06003212c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -44,7 +44,7 @@ findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} flink120 = { strictly = "1.20.0"} -google-libraries-bom = "26.48.0" +google-libraries-bom = "26.49.0" guava = "33.3.1-jre" hadoop2 = "2.7.3" hadoop3 = "3.3.6" From c16cefa5015dda417f80e0d59124cd92448787ab Mon Sep 17 00:00:00 2001 From: leesf <490081539@qq.com> Date: Mon, 21 Oct 2024 20:35:03 +0800 Subject: [PATCH 026/313] Core: Move deleteRemovedMetadataFiles(..) to CatalogUtil (#11352) --- .../iceberg/BaseMetastoreTableOperations.java | 49 +------------------ .../java/org/apache/iceberg/CatalogUtil.java | 45 +++++++++++++++++ .../iceberg/hadoop/HadoopTableOperations.java | 40 +-------------- 3 files changed, 48 insertions(+), 86 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 53f3250dc95c..dbab9e813966 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -19,7 +19,6 @@ package org.apache.iceberg; import java.util.Locale; -import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -32,11 +31,8 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.LocationUtil; import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; @@ -127,7 +123,7 @@ public void commit(TableMetadata base, TableMetadata metadata) { long start = System.currentTimeMillis(); doCommit(base, metadata); - deleteRemovedMetadataFiles(base, metadata); + CatalogUtil.deleteRemovedMetadataFiles(io(), base, metadata); requestRefresh(); LOG.info( @@ -354,47 +350,4 @@ private static int parseVersion(String metadataLocation) { return -1; } } - - /** - * Deletes the oldest metadata files if {@link - * TableProperties#METADATA_DELETE_AFTER_COMMIT_ENABLED} is true. - * - * @param base table metadata on which previous versions were based - * @param metadata new table metadata with updated previous versions - */ - private void deleteRemovedMetadataFiles(TableMetadata base, TableMetadata metadata) { - if (base == null) { - return; - } - - boolean deleteAfterCommit = - metadata.propertyAsBoolean( - TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, - TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT); - - if (deleteAfterCommit) { - Set removedPreviousMetadataFiles = - Sets.newHashSet(base.previousFiles()); - // TableMetadata#addPreviousFile builds up the metadata log and uses - // TableProperties.METADATA_PREVIOUS_VERSIONS_MAX to determine how many files should stay in - // the log, thus we don't include metadata.previousFiles() for deletion - everything else can - // be removed - removedPreviousMetadataFiles.removeAll(metadata.previousFiles()); - if (io() instanceof SupportsBulkOperations) { - ((SupportsBulkOperations) io()) - .deleteFiles( - Iterables.transform( - removedPreviousMetadataFiles, TableMetadata.MetadataLogEntry::file)); - } else { - Tasks.foreach(removedPreviousMetadataFiles) - .noRetry() - .suppressFailureWhenFinished() - .onFailure( - (previousMetadataFile, exc) -> - LOG.warn( - "Delete failed for previous metadata file: {}", previousMetadataFile, exc)) - .run(previousMetadataFile -> io().deleteFile(previousMetadataFile.file())); - } - } - } } diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 70b10cbaeb62..609e94b7b150 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -515,4 +515,49 @@ public static String fullTableName(String catalogName, TableIdentifier identifie return sb.toString(); } + + /** + * Deletes the oldest metadata files if {@link + * TableProperties#METADATA_DELETE_AFTER_COMMIT_ENABLED} is true. + * + * @param io FileIO instance to use for deletes + * @param base table metadata on which previous versions were based + * @param metadata new table metadata with updated previous versions + */ + public static void deleteRemovedMetadataFiles( + FileIO io, TableMetadata base, TableMetadata metadata) { + if (base == null) { + return; + } + + boolean deleteAfterCommit = + metadata.propertyAsBoolean( + TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, + TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT); + + if (deleteAfterCommit) { + Set removedPreviousMetadataFiles = + Sets.newHashSet(base.previousFiles()); + // TableMetadata#addPreviousFile builds up the metadata log and uses + // TableProperties.METADATA_PREVIOUS_VERSIONS_MAX to determine how many files should stay in + // the log, thus we don't include metadata.previousFiles() for deletion - everything else can + // be removed + removedPreviousMetadataFiles.removeAll(metadata.previousFiles()); + if (io instanceof SupportsBulkOperations) { + ((SupportsBulkOperations) io) + .deleteFiles( + Iterables.transform( + removedPreviousMetadataFiles, TableMetadata.MetadataLogEntry::file)); + } else { + Tasks.foreach(removedPreviousMetadataFiles) + .noRetry() + .suppressFailureWhenFinished() + .onFailure( + (previousMetadataFile, exc) -> + LOG.warn( + "Delete failed for previous metadata file: {}", previousMetadataFile, exc)) + .run(previousMetadataFile -> io.deleteFile(previousMetadataFile.file())); + } + } + } } diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java index 1e0cf4422120..24299371401c 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; -import java.util.Set; import java.util.UUID; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -31,6 +30,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.LocationProviders; import org.apache.iceberg.LockManager; import org.apache.iceberg.TableMetadata; @@ -45,10 +45,7 @@ import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.Pair; -import org.apache.iceberg.util.Tasks; -import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -168,7 +165,7 @@ public void commit(TableMetadata base, TableMetadata metadata) { // update the best-effort version pointer writeVersionHint(nextVersion); - deleteRemovedMetadataFiles(base, metadata); + CatalogUtil.deleteRemovedMetadataFiles(io(), base, metadata); this.shouldRefresh = true; } @@ -414,39 +411,6 @@ protected FileSystem getFileSystem(Path path, Configuration hadoopConf) { return Util.getFs(path, hadoopConf); } - /** - * Deletes the oldest metadata files if {@link - * TableProperties#METADATA_DELETE_AFTER_COMMIT_ENABLED} is true. - * - * @param base table metadata on which previous versions were based - * @param metadata new table metadata with updated previous versions - */ - private void deleteRemovedMetadataFiles(TableMetadata base, TableMetadata metadata) { - if (base == null) { - return; - } - - boolean deleteAfterCommit = - metadata.propertyAsBoolean( - TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED, - TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT); - - if (deleteAfterCommit) { - Set removedPreviousMetadataFiles = - Sets.newHashSet(base.previousFiles()); - removedPreviousMetadataFiles.removeAll(metadata.previousFiles()); - Tasks.foreach(removedPreviousMetadataFiles) - .executeWith(ThreadPools.getWorkerPool()) - .noRetry() - .suppressFailureWhenFinished() - .onFailure( - (previousMetadataFile, exc) -> - LOG.warn( - "Delete failed for previous metadata file: {}", previousMetadataFile, exc)) - .run(previousMetadataFile -> io().deleteFile(previousMetadataFile.file())); - } - } - private static TableMetadata checkUUID(TableMetadata currentMetadata, TableMetadata newMetadata) { String newUUID = newMetadata.uuid(); if (currentMetadata != null && currentMetadata.uuid() != null && newUUID != null) { From d0a7ff915e6307e200a02ed76daff1a2fb363a3b Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Mon, 21 Oct 2024 11:53:13 -0700 Subject: [PATCH 027/313] Arrow: Fix indexing in Parquet dictionary encoded values readers (#11247) --- ...dDictionaryEncodedParquetValuesReader.java | 22 ++--- .../iceberg/spark/data/TestHelpers.java | 15 +++ ...rquetDictionaryEncodedVectorizedReads.java | 92 ++++++++++++++++++ .../decimal_dict_and_plain_encoding.parquet | Bin 0 -> 3685 bytes 4 files changed, 116 insertions(+), 13 deletions(-) create mode 100644 spark/v3.5/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java index 4499d0536867..848bb2d936c3 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java @@ -58,14 +58,10 @@ public void nextBatch( } int numValues = Math.min(left, currentCount); for (int i = 0; i < numValues; i++) { - int index = idx * typeWidth; - if (typeWidth == -1) { - index = idx; - } if (Mode.RLE.equals(mode)) { - nextVal(vector, dict, index, currentValue, typeWidth); + nextVal(vector, dict, idx, currentValue, typeWidth); } else if (Mode.PACKED.equals(mode)) { - nextVal(vector, dict, index, packedValuesBuffer[packedValuesBufferIdx++], typeWidth); + nextVal(vector, dict, idx, packedValuesBuffer[packedValuesBufferIdx++], typeWidth); } nullabilityHolder.setNotNull(idx); if (setArrowValidityVector) { @@ -94,7 +90,7 @@ class LongDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentVal)); + vector.getDataBuffer().setLong((long) idx * typeWidth, dict.decodeToLong(currentVal)); } } @@ -102,7 +98,7 @@ class TimestampMillisDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentVal) * 1000); + vector.getDataBuffer().setLong((long) idx * typeWidth, dict.decodeToLong(currentVal) * 1000); } } @@ -113,7 +109,7 @@ protected void nextVal( ByteBuffer buffer = dict.decodeToBinary(currentVal).toByteBuffer().order(ByteOrder.LITTLE_ENDIAN); long timestampInt96 = ParquetUtil.extractTimestampInt96(buffer); - vector.getDataBuffer().setLong(idx, timestampInt96); + vector.getDataBuffer().setLong((long) idx * typeWidth, timestampInt96); } } @@ -121,7 +117,7 @@ class IntegerDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setInt(idx, dict.decodeToInt(currentVal)); + vector.getDataBuffer().setInt((long) idx * typeWidth, dict.decodeToInt(currentVal)); } } @@ -129,7 +125,7 @@ class FloatDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentVal)); + vector.getDataBuffer().setFloat((long) idx * typeWidth, dict.decodeToFloat(currentVal)); } } @@ -137,7 +133,7 @@ class DoubleDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentVal)); + vector.getDataBuffer().setDouble((long) idx * typeWidth, dict.decodeToDouble(currentVal)); } } @@ -150,7 +146,7 @@ class FixedWidthBinaryDictEncodedReader extends BaseDictEncodedReader { protected void nextVal( FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { ByteBuffer buffer = dict.decodeToBinary(currentVal).toByteBuffer(); - vector.getDataBuffer().setBytes(idx, buffer); + vector.getDataBuffer().setBytes((long) idx * typeWidth, buffer); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 4252838d5f53..d64ca588f202 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -116,6 +116,21 @@ public static void assertEqualsBatch( } } + public static void assertEqualsBatchWithRows( + Types.StructType struct, Iterator expected, ColumnarBatch batch) { + for (int rowId = 0; rowId < batch.numRows(); rowId++) { + List fields = struct.fields(); + InternalRow row = batch.getRow(rowId); + Row expectedRow = expected.next(); + for (int i = 0; i < fields.size(); i += 1) { + Type fieldType = fields.get(i).type(); + Object expectedValue = expectedRow.get(i); + Object actualValue = row.isNullAt(i) ? null : row.get(i, convert(fieldType)); + assertEqualsUnsafe(fieldType, expectedValue, actualValue); + } + } + } + private static void assertEqualsSafe(Types.ListType list, Collection expected, List actual) { Type elementType = list.elementType(); List expectedElements = Lists.newArrayList(expected); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java index eeed9d1a03ce..bc4e722bc869 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetDictionaryEncodedVectorizedReads.java @@ -18,13 +18,21 @@ */ package org.apache.iceberg.spark.data.parquet.vectorized; +import static org.apache.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.PARQUET_PAGE_ROW_LIMIT; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Iterator; +import java.util.List; import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Function; @@ -33,11 +41,35 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; public class TestParquetDictionaryEncodedVectorizedReads extends TestParquetVectorizedReads { + protected static SparkSession spark = null; + + @BeforeAll + public static void startSpark() { + spark = SparkSession.builder().master("local[2]").getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + if (spark != null) { + spark.stop(); + spark = null; + } + } + @Override Iterable generateData( Schema schema, @@ -93,4 +125,64 @@ public void testMixedDictionaryNonDictionaryReads() throws IOException { true, BATCH_SIZE); } + + @Test + public void testBinaryNotAllPagesDictionaryEncoded() throws IOException { + Schema schema = new Schema(Types.NestedField.required(1, "bytes", Types.BinaryType.get())); + File parquetFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(parquetFile.delete()).as("Delete should succeed").isTrue(); + + Iterable records = RandomData.generateFallbackData(schema, 500, 0L, 100); + try (FileAppender writer = + Parquet.write(Files.localOutput(parquetFile)) + .schema(schema) + .set(PARQUET_DICT_SIZE_BYTES, "4096") + .set(PARQUET_PAGE_ROW_LIMIT, "100") + .build()) { + writer.addAll(records); + } + + // After the above, parquetFile contains one column chunk of binary data in five pages, + // the first two RLE dictionary encoded, and the remaining three plain encoded. + assertRecordsMatch(schema, 500, records, parquetFile, true, BATCH_SIZE); + } + + /** + * decimal_dict_and_plain_encoding.parquet contains one column chunk of decimal(38, 0) data in two + * pages, one RLE dictionary encoded and one plain encoded, each with 200 rows. + */ + @Test + public void testDecimalNotAllPagesDictionaryEncoded() throws Exception { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.DecimalType.of(38, 0))); + Path path = + Paths.get( + getClass() + .getClassLoader() + .getResource("decimal_dict_and_plain_encoding.parquet") + .toURI()); + + Dataset df = spark.read().parquet(path.toString()); + List expected = df.collectAsList(); + long expectedSize = df.count(); + + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(path.toFile())) + .project(schema) + .createBatchedReaderFunc( + type -> + VectorizedSparkParquetReaders.buildReader( + schema, type, ImmutableMap.of(), null)); + + try (CloseableIterable batchReader = readBuilder.build()) { + Iterator expectedIter = expected.iterator(); + Iterator batches = batchReader.iterator(); + int numRowsRead = 0; + while (batches.hasNext()) { + ColumnarBatch batch = batches.next(); + numRowsRead += batch.numRows(); + TestHelpers.assertEqualsBatchWithRows(schema.asStruct(), expectedIter, batch); + } + assertThat(numRowsRead).isEqualTo(expectedSize); + } + } } diff --git a/spark/v3.5/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet b/spark/v3.5/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet new file mode 100644 index 0000000000000000000000000000000000000000..48b3bd1bf24f13150e2e283bf61f5d4776c3754a GIT binary patch literal 3685 zcmd6qd2~}%9>-tuZt`xEv<-!nq@;^o>6Ruby``mT`%+rcjV5K6tp#eKGlfu=0%a)( zRAB~!tm0M?1=*&`fFKCP3dq433wj2IL8gf4IG_U#=n<4N_xo_pnWOySpY#66=lz!Z zyZi6=;+3l%CSq!JnjZA2Fa<=D5GDw%7D2G71ZwNQ)fOrk1XJxnr%|BNK_4>;#P!2{NXlWm^%kBUmVLpGMubKMf)%R3jTR1T`)!6{tyUK4+Zqi z`vLjKaX?(+XIP>WDG;jFUjilNK|p9r8xRqZ1^6bG0IoYn0C8~vP~AHiFfUF6k}lr^ zq}Y)_@>g08l75v1RMd3$4Gd_&hRe_y>RF9Q4XqQQ3(d`kuDF94)im4o4s=Cl%m|^2 z8VzKg90J6jqd;n9F;KZk0*u#o0Md8=0BqCZfb@^tK{Hwxg z=d*tE8Z5C}S&fP$oMm1$4cb)7Eu?BR*1PKXG(HSs+|O>QXvVjnVoaW|zd_Itzl^5l z3U*6PLr3?9X~+&0Oc7UR0Q%uc@X$TSI@KgZWUYj6)n0V#cv%exx*!s8RR@5SlD`AS zq?dt=70rNnu@6wKJ^=V8MgWm}@;Hb-zzVgrXhtTwcB%gaT~!NK~Eod$f34M0Dk9B_1HaFFO5tX)l`;?vQC{l+5b zbd5|<5$AjsBNBZv-X$Mv*U;1()16)(;H`@Y5ci653S(EZ8V#kMMg)*lFT#|6X(`a_ zQYFGwS=cR+->rCfB%GZKUHmo1g_42Ka~&BVGeN^Q>Nte#`^>H(kDX?%{IFTaV}7NZ zSdW@-_!0!jS!73m3>k;tcw`Kr?5gw^!u4tBoR40 zFdfiqveVBO#?N|0T7148%{^c5L-VwX+*=~q{~A=$88@H`apRbbO9DA$K2e9j)H}E=-}htX)O2&#-op`l4RJ z+BS{5!^yrgyM zvL~0XSlRZ})2mkh?)Pif2G%{Ze#6F1o3}i>^||L?c=4t7m$$vL{ngiY?0kLK?mc_= z{h{NHH~0VXz`?f;9e(@B(RYp=?>zBt*Lx>Vo#uVeErO5%2J_!d&} z$wr``7Po?yEHMCms+9649yLM=V@kS}I^rSO2PzdS@mdZf{U#;(W-eO|DeJx)FvN@p zd}A_!YVtf#O4b57v>UM9_W|WIm3k>H8xAS|m{Lo79V>-2bcB-YQU@w|DaE)ON=M;! zK!0ik5a<5@aNbi2W%ckAkcxK{a**)jJQ>lxn?J$9*WN?^q4`1KfT30?ZY!1IHl%9| z+}Rh*t|HzG%*zw;swiA1$3dk&%5MkI$l9gcPAOg~B~^cOKsof^lL61FUC5w~%7=le zk8%t9u0*mdzo0??2FGFe#Xe99Y4C#tNF_%`0zS_LAY|}hU~qCBpl`bgBzTSjnX{E5 zkmss~)ccZ`gLqu5hgXDoW#}M#Zz>;wP{#Wtp@)&;gFj#$Y8p}SF)p@lffL_9?n#%O zy?EjO*oHXK{oVws4qJhW&y@OayLb}82TkDFho-M+KyY#I9e50A|0_&!!&o<=hG83E zN^H)DDXjS_AcYSDjMvWtIV~2zbLbLKNxuN3x>FeFGUV6NYmtcI8-w~tf+-6KB(shu zbE4nGIz-|y@?t<-!1% zU3xE6lk6Nn*OB5K%&w*e|2PN{yO~kLk5LeWgdof)^ZY6_r3rZhQY4j@T4fUu>T z5ieyUi_wy_n=GIq;SH=^BECUPkZ98D5h!+vPppJ3Mz3g3lL(O zixXD%LLyYQ0#?YY=z=OlkxE#hhPsl{xIn{_dN7Wm>%uS&8j8W&e_)*&>Ymh!rn&`) zqd(9IZ|{M6c=zjzO~X;*D`Vb!1X0r>m|w>?cMbfc@K9{TZRVxa)jSibq=|O4aXayW zqf23eo@C8nenNv=wmJ)W*%`(HYEo`zc9E2sSr0z|+qT0nC=X+ox|DB0DSio}RPJLn zl=kmhgvgGA>!B<9kVnHEwM)cnXMrGU3r^wzXu9_)Zl}3}@$Eti+E|B%SZ6aoKS*u- zbAiPD1^ZTV1Y2NmTu;1cx0*sMS%TJRF{`IdF{#7bwc+O*!n^H>J57o7S5&!2CG20; z!g*zaC6E6r5lm`<|NI0amjuB?ENNMSSvza;)Vju6IE0vDOlpiLLC0My(#GMJd-B0q ztQPD4#i+4L?b<}jm#U8-SgdA)TwPr?D)-?m%zIQBcrxo{{`m02pK@7lmfiJoi7eO4 zMT$vwyJfi&#v)lR1-blRa;X@q5%}S6ob_n$K^*@0;Qa6@_9&iiMX$U*)FYJJ|M){; zh|*q`Ic2%*VgGn_3Fa7oZPUy-b+gka`~B&)O-(bJW9HU1&6+l2dW<96X1CcbRx96u MJt{%)@;{jW0iIk85dZ)H literal 0 HcmV?d00001 From a19896658c98220ea0115d5d93ba6dff40c5c4e9 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Tue, 22 Oct 2024 09:13:01 -0700 Subject: [PATCH 028/313] Spark 3.5: Update Spark to use planned Avro reads (#11299) --- .../iceberg/avro/AvroWithPartnerVisitor.java | 2 +- .../org/apache/iceberg/avro/ValueReaders.java | 4 +- .../iceberg/spark/data/SparkAvroReader.java | 12 ++ .../spark/data/SparkPlannedAvroReader.java | 190 ++++++++++++++++++ .../iceberg/spark/data/SparkValueReaders.java | 38 ++++ .../iceberg/spark/source/BaseRowReader.java | 4 +- .../spark/data/TestSparkAvroEnums.java | 2 +- .../spark/data/TestSparkAvroReader.java | 2 +- .../spark/source/TestDataFrameWrites.java | 4 +- 9 files changed, 249 insertions(+), 9 deletions(-) create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java index b23b195d959a..692c1ead3fbf 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java @@ -37,7 +37,7 @@ public interface PartnerAccessors

{ P listElementPartner(P partnerList); } - static class FieldIDAccessors implements AvroWithPartnerVisitor.PartnerAccessors { + public static class FieldIDAccessors implements AvroWithPartnerVisitor.PartnerAccessors { private static final FieldIDAccessors INSTANCE = new FieldIDAccessors(); public static FieldIDAccessors get() { diff --git a/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java index 246671076c31..67f53d3636a6 100644 --- a/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java @@ -194,11 +194,11 @@ public static ValueReader skipStruct(List> readers) { * @param idToConstant a map of field ID to constants values * @return a read plan that is a list of (position, reader) pairs */ - static List>> buildReadPlan( + public static List>> buildReadPlan( Types.StructType expected, Schema record, List> fieldReaders, - Map idToConstant) { + Map idToConstant) { Map idToPos = idToPos(expected); List>> readPlan = Lists.newArrayList(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java index 4622d2928ac4..7d92d963a9f4 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java @@ -37,16 +37,28 @@ import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.InternalRow; +/** + * @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead. + */ +@Deprecated public class SparkAvroReader implements DatumReader, SupportsRowPosition { private final Schema readSchema; private final ValueReader reader; private Schema fileSchema = null; + /** + * @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead. + */ + @Deprecated public SparkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { this(expectedSchema, readSchema, ImmutableMap.of()); } + /** + * @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead. + */ + @Deprecated @SuppressWarnings("unchecked") public SparkAvroReader( org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java new file mode 100644 index 000000000000..dc4af24685b3 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.data; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.iceberg.avro.AvroWithPartnerVisitor; +import org.apache.iceberg.avro.SupportsRowPosition; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; + +public class SparkPlannedAvroReader implements DatumReader, SupportsRowPosition { + + private final Types.StructType expectedType; + private final Map idToConstant; + private ValueReader reader; + + public static SparkPlannedAvroReader create(org.apache.iceberg.Schema schema) { + return create(schema, ImmutableMap.of()); + } + + public static SparkPlannedAvroReader create( + org.apache.iceberg.Schema schema, Map constants) { + return new SparkPlannedAvroReader(schema, constants); + } + + private SparkPlannedAvroReader( + org.apache.iceberg.Schema expectedSchema, Map constants) { + this.expectedType = expectedSchema.asStruct(); + this.idToConstant = constants; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema fileSchema) { + this.reader = + (ValueReader) + AvroWithPartnerVisitor.visit( + expectedType, + fileSchema, + new ReadBuilder(idToConstant), + AvroWithPartnerVisitor.FieldIDAccessors.get()); + } + + @Override + public InternalRow read(InternalRow reuse, Decoder decoder) throws IOException { + return reader.read(decoder, reuse); + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + private static class ReadBuilder extends AvroWithPartnerVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public ValueReader record(Type partner, Schema record, List> fieldReaders) { + if (partner == null) { + return ValueReaders.skipStruct(fieldReaders); + } + + Types.StructType expected = partner.asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldReaders, idToConstant); + + // TODO: should this pass expected so that struct.get can reuse containers? + return SparkValueReaders.struct(readPlan, expected.fields().size()); + } + + @Override + public ValueReader union(Type partner, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader array(Type partner, Schema array, ValueReader elementReader) { + return SparkValueReaders.array(elementReader); + } + + @Override + public ValueReader arrayMap( + Type partner, Schema map, ValueReader keyReader, ValueReader valueReader) { + return SparkValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader map(Type partner, Schema map, ValueReader valueReader) { + return SparkValueReaders.map(SparkValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Type partner, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + // Spark uses the same representation + return ValueReaders.ints(); + + case "timestamp-millis": + // adjust to microseconds + ValueReader longs = ValueReaders.longs(); + return (ValueReader) (decoder, ignored) -> longs.read(decoder, null) * 1000L; + + case "timestamp-micros": + // Spark uses the same representation + return ValueReaders.longs(); + + case "decimal": + return SparkValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + ((LogicalTypes.Decimal) logicalType).getScale()); + + case "uuid": + return SparkValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + if (partner != null && partner.typeId() == Type.TypeID.LONG) { + return ValueReaders.intsAsLongs(); + } + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + if (partner != null && partner.typeId() == Type.TypeID.DOUBLE) { + return ValueReaders.floatsAsDoubles(); + } + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return SparkValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive.getFixedSize()); + case BYTES: + return ValueReaders.bytes(); + case ENUM: + return SparkValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java index 3cbf38d88bf4..7e65535f5ecb 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java @@ -32,6 +32,7 @@ import org.apache.iceberg.avro.ValueReaders; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.UUIDUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; @@ -74,6 +75,11 @@ static ValueReader map(ValueReader keyReader, ValueReader< return new MapReader(keyReader, valueReader); } + static ValueReader struct( + List>> readPlan, int numFields) { + return new PlannedStructReader(readPlan, numFields); + } + static ValueReader struct( List> readers, Types.StructType struct, Map idToConstant) { return new StructReader(readers, struct, idToConstant); @@ -249,6 +255,38 @@ public ArrayBasedMapData read(Decoder decoder, Object reuse) throws IOException } } + static class PlannedStructReader extends ValueReaders.PlannedStructReader { + private final int numFields; + + protected PlannedStructReader(List>> readPlan, int numFields) { + super(readPlan); + this.numFields = numFields; + } + + @Override + protected InternalRow reuseOrCreate(Object reuse) { + if (reuse instanceof GenericInternalRow + && ((GenericInternalRow) reuse).numFields() == numFields) { + return (InternalRow) reuse; + } + return new GenericInternalRow(numFields); + } + + @Override + protected Object get(InternalRow struct, int pos) { + return null; + } + + @Override + protected void set(InternalRow struct, int pos, Object value) { + if (value != null) { + struct.update(pos, value); + } else { + struct.setNullAt(pos); + } + } + } + static class StructReader extends ValueReaders.StructReader { private final int numFields; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java index 927084caea1c..eb97185e21f1 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java @@ -32,9 +32,9 @@ import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.spark.data.SparkAvroReader; import org.apache.iceberg.spark.data.SparkOrcReader; import org.apache.iceberg.spark.data.SparkParquetReaders; +import org.apache.iceberg.spark.data.SparkPlannedAvroReader; import org.apache.iceberg.types.TypeUtil; import org.apache.spark.sql.catalyst.InternalRow; @@ -77,7 +77,7 @@ private CloseableIterable newAvroIterable( .reuseContainers() .project(projection) .split(start, length) - .createReaderFunc(readSchema -> new SparkAvroReader(projection, readSchema, idToConstant)) + .createReaderFunc(readSchema -> SparkPlannedAvroReader.create(projection, idToConstant)) .withNameMapping(nameMapping()) .build(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java index 11e60187fdc3..0dc8b48b2317 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java @@ -79,7 +79,7 @@ public void writeAndValidateEnums() throws IOException { List rows; try (AvroIterable reader = Avro.read(Files.localInput(testFile)) - .createReaderFunc(SparkAvroReader::new) + .createResolvingReader(SparkPlannedAvroReader::create) .project(schema) .build()) { rows = Lists.newArrayList(reader); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java index 3e5088258a49..7f9bcbacf298 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java @@ -51,7 +51,7 @@ protected void writeAndValidate(Schema schema) throws IOException { List rows; try (AvroIterable reader = Avro.read(Files.localInput(testFile)) - .createReaderFunc(SparkAvroReader::new) + .createResolvingReader(SparkPlannedAvroReader::create) .project(schema) .build()) { rows = Lists.newArrayList(reader); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 336ee5a8d2ea..bf49bfba550f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -56,7 +56,7 @@ import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.iceberg.spark.data.ParameterizedAvroDataTest; import org.apache.iceberg.spark.data.RandomData; -import org.apache.iceberg.spark.data.SparkAvroReader; +import org.apache.iceberg.spark.data.SparkPlannedAvroReader; import org.apache.iceberg.types.Types; import org.apache.spark.SparkException; import org.apache.spark.TaskContext; @@ -259,7 +259,7 @@ private Dataset createDataset(Iterable records, Schema schema) thro List rows = Lists.newArrayList(); try (AvroIterable reader = Avro.read(Files.localInput(testFile)) - .createReaderFunc(SparkAvroReader::new) + .createResolvingReader(SparkPlannedAvroReader::create) .project(schema) .build()) { From a8ec43d975d8d3bbacb6880b487208b00cb7361f Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Tue, 22 Oct 2024 15:28:57 -0700 Subject: [PATCH 029/313] Core, Spark 3.5: Remove dangling deletes as part of RewriteDataFilesAction (#9724) --- .../iceberg/actions/ActionsProvider.java | 6 + .../actions/RemoveDanglingDeleteFiles.java | 35 ++ .../iceberg/actions/RewriteDataFiles.java | 16 + .../BaseRemoveDanglingDeleteFiles.java | 33 ++ .../iceberg/actions/BaseRewriteDataFiles.java | 6 + .../iceberg/spark/SparkContentFile.java | 7 +- .../RemoveDanglingDeletesSparkAction.java | 179 +++++++ .../actions/RewriteDataFilesSparkAction.java | 34 +- .../iceberg/spark/actions/SparkActions.java | 6 + .../TestRemoveDanglingDeleteAction.java | 447 ++++++++++++++++++ .../actions/TestRewriteDataFilesAction.java | 215 ++++++++- .../TestRewritePositionDeleteFilesAction.java | 3 +- 12 files changed, 974 insertions(+), 13 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/actions/RemoveDanglingDeleteFiles.java create mode 100644 core/src/main/java/org/apache/iceberg/actions/BaseRemoveDanglingDeleteFiles.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java diff --git a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java index bcc77b25d698..61750d83fc79 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -82,4 +82,10 @@ default RewriteTablePath rewriteTablePath(Table table) { throw new UnsupportedOperationException( this.getClass().getName() + " does not implement rewriteTablePath"); } + + /** Instantiates an action to remove dangling delete files from current snapshot. */ + default RemoveDanglingDeleteFiles removeDanglingDeleteFiles(Table table) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement removeDanglingDeleteFiles"); + } } diff --git a/api/src/main/java/org/apache/iceberg/actions/RemoveDanglingDeleteFiles.java b/api/src/main/java/org/apache/iceberg/actions/RemoveDanglingDeleteFiles.java new file mode 100644 index 000000000000..b0ef0d5e35f8 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/actions/RemoveDanglingDeleteFiles.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import org.apache.iceberg.DeleteFile; + +/** + * An action that removes dangling delete files from the current snapshot. A delete file is dangling + * if its deletes no longer applies to any live data files. + */ +public interface RemoveDanglingDeleteFiles + extends Action { + + /** An action that remove dangling deletes. */ + interface Result { + /** Return removed deletes. */ + Iterable removedDeleteFiles(); + } +} diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java b/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java index f6ef40270852..589b9017741e 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java @@ -106,6 +106,18 @@ public interface RewriteDataFiles boolean USE_STARTING_SEQUENCE_NUMBER_DEFAULT = true; + /** + * Remove dangling delete files from the current snapshot after compaction. A delete file is + * considered dangling if it does not apply to any live data files. + * + *

Both equality and position dangling delete files will be removed. + * + *

Defaults to false. + */ + String REMOVE_DANGLING_DELETES = "remove-dangling-deletes"; + + boolean REMOVE_DANGLING_DELETES_DEFAULT = false; + /** * Forces the rewrite job order based on the value. * @@ -216,6 +228,10 @@ default long rewrittenBytesCount() { default int failedDataFilesCount() { return rewriteFailures().stream().mapToInt(FileGroupFailureResult::dataFilesCount).sum(); } + + default int removedDeleteFilesCount() { + return 0; + } } /** diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRemoveDanglingDeleteFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseRemoveDanglingDeleteFiles.java new file mode 100644 index 000000000000..3b5ce9e79a43 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRemoveDanglingDeleteFiles.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import org.immutables.value.Value; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableRemoveDanglingDeleteFiles", + visibilityString = "PUBLIC", + builderVisibilityString = "PUBLIC") +interface BaseRemoveDanglingDeleteFiles extends RemoveDanglingDeleteFiles { + + @Value.Immutable + interface Result extends RemoveDanglingDeleteFiles.Result {} +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java index 953439484a15..2faa1f1b756c 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFiles.java @@ -55,6 +55,12 @@ default long rewrittenBytesCount() { return RewriteDataFiles.Result.super.rewrittenBytesCount(); } + @Override + @Value.Default + default int removedDeleteFilesCount() { + return RewriteDataFiles.Result.super.removedDeleteFilesCount(); + } + @Override @Value.Default default int failedDataFilesCount() { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java index f756c4cde015..99586f2503c2 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java @@ -52,6 +52,7 @@ public abstract class SparkContentFile implements ContentFile { private final int keyMetadataPosition; private final int splitOffsetsPosition; private final int sortOrderIdPosition; + private final int fileSpecIdPosition; private final int equalityIdsPosition; private final Type lowerBoundsType; private final Type upperBoundsType; @@ -100,6 +101,7 @@ public abstract class SparkContentFile implements ContentFile { this.keyMetadataPosition = positions.get(DataFile.KEY_METADATA.name()); this.splitOffsetsPosition = positions.get(DataFile.SPLIT_OFFSETS.name()); this.sortOrderIdPosition = positions.get(DataFile.SORT_ORDER_ID.name()); + this.fileSpecIdPosition = positions.get(DataFile.SPEC_ID.name()); this.equalityIdsPosition = positions.get(DataFile.EQUALITY_IDS.name()); } @@ -120,7 +122,10 @@ public Long pos() { @Override public int specId() { - return -1; + if (wrapped.isNullAt(fileSpecIdPosition)) { + return -1; + } + return wrapped.getAs(fileSpecIdPosition); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java new file mode 100644 index 000000000000..bbf65f58e19c --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.min; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ImmutableRemoveDanglingDeleteFiles; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.SparkDeleteFile; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An action that removes dangling delete files from the current snapshot. A delete file is dangling + * if its deletes no longer applies to any live data files. + * + *

The following dangling delete files are removed: + * + *

    + *
  • Position delete files with a data sequence number less than that of any data file in the + * same partition + *
  • Equality delete files with a data sequence number less than or equal to that of any data + * file in the same partition + *
+ */ +class RemoveDanglingDeletesSparkAction + extends BaseSnapshotUpdateSparkAction + implements RemoveDanglingDeleteFiles { + + private static final Logger LOG = LoggerFactory.getLogger(RemoveDanglingDeletesSparkAction.class); + private final Table table; + + protected RemoveDanglingDeletesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + } + + @Override + protected RemoveDanglingDeletesSparkAction self() { + return this; + } + + public Result execute() { + if (table.specs().size() == 1 && table.spec().isUnpartitioned()) { + // ManifestFilterManager already performs this table-wide delete on each commit + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(Collections.emptyList()) + .build(); + } + + String desc = String.format("Removing dangling delete files in %s", table.name()); + JobGroupInfo info = newJobGroupInfo("REMOVE-DELETES", desc); + return withJobGroupInfo(info, this::doExecute); + } + + Result doExecute() { + RewriteFiles rewriteFiles = table.newRewrite(); + List danglingDeletes = findDanglingDeletes(); + for (DeleteFile deleteFile : danglingDeletes) { + LOG.debug("Removing dangling delete file {}", deleteFile.path()); + rewriteFiles.deleteFile(deleteFile); + } + + if (!danglingDeletes.isEmpty()) { + commit(rewriteFiles); + } + + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(danglingDeletes) + .build(); + } + + /** + * Dangling delete files can be identified with following steps + * + *
    + *
  1. Group data files by partition keys and find the minimum data sequence number in each + * group. + *
  2. Left outer join delete files with partition-grouped data files on partition keys. + *
  3. Find dangling deletes by comparing each delete file's sequence number to its partition's + * minimum data sequence number. + *
  4. Collect results row to driver and use {@link SparkDeleteFile SparkDeleteFile} to wrap + * rows to valid delete files + *
+ */ + private List findDanglingDeletes() { + Dataset minSequenceNumberByPartition = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live data files + .filter("data_file.content == 0 AND status < 2") + .selectExpr( + "data_file.partition as partition", + "data_file.spec_id as spec_id", + "sequence_number") + .groupBy("partition", "spec_id") + .agg(min("sequence_number")) + .toDF("grouped_partition", "grouped_spec_id", "min_data_sequence_number"); + + Dataset deleteEntries = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live delete files + .filter("data_file.content != 0 AND status < 2"); + + Column joinOnPartition = + deleteEntries + .col("data_file.spec_id") + .equalTo(minSequenceNumberByPartition.col("grouped_spec_id")) + .and( + deleteEntries + .col("data_file.partition") + .equalTo(minSequenceNumberByPartition.col("grouped_partition"))); + + Column filterOnDanglingDeletes = + col("min_data_sequence_number") + // delete fies without any data files in partition + .isNull() + // position delete files without any applicable data files in partition + .or( + col("data_file.content") + .equalTo("1") + .and(col("sequence_number").$less(col("min_data_sequence_number")))) + // equality delete files without any applicable data files in the partition + .or( + col("data_file.content") + .equalTo("2") + .and(col("sequence_number").$less$eq(col("min_data_sequence_number")))); + + Dataset danglingDeletes = + deleteEntries + .join(minSequenceNumberByPartition, joinOnPartition, "left") + .filter(filterOnDanglingDeletes) + .select("data_file.*"); + return danglingDeletes.collectAsList().stream() + // map on driver because SparkDeleteFile is not serializable + .map(row -> deleteFileWrapper(danglingDeletes.schema(), row)) + .collect(Collectors.toList()); + } + + private DeleteFile deleteFileWrapper(StructType sparkFileType, Row row) { + int specId = row.getInt(row.fieldIndex("spec_id")); + Types.StructType combinedFileType = DataFile.getType(Partitioning.partitionType(table)); + // Set correct spec id + Types.StructType projection = DataFile.getType(table.specs().get(specId).partitionType()); + return new SparkDeleteFile(combinedFileType, projection, sparkFileType).wrap(row); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java index d33e5e540893..4e381a7bd362 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java @@ -40,6 +40,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.FileRewriter; import org.apache.iceberg.actions.ImmutableRewriteDataFiles; +import org.apache.iceberg.actions.ImmutableRewriteDataFiles.Result.Builder; import org.apache.iceberg.actions.RewriteDataFiles; import org.apache.iceberg.actions.RewriteDataFilesCommitManager; import org.apache.iceberg.actions.RewriteFileGroup; @@ -53,6 +54,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Queues; @@ -83,7 +85,8 @@ public class RewriteDataFilesSparkAction TARGET_FILE_SIZE_BYTES, USE_STARTING_SEQUENCE_NUMBER, REWRITE_JOB_ORDER, - OUTPUT_SPEC_ID); + OUTPUT_SPEC_ID, + REMOVE_DANGLING_DELETES); private static final RewriteDataFilesSparkAction.Result EMPTY_RESULT = ImmutableRewriteDataFiles.Result.builder().rewriteResults(ImmutableList.of()).build(); @@ -95,6 +98,7 @@ public class RewriteDataFilesSparkAction private int maxCommits; private int maxFailedCommits; private boolean partialProgressEnabled; + private boolean removeDanglingDeletes; private boolean useStartingSequenceNumber; private RewriteJobOrder rewriteJobOrder; private FileRewriter rewriter = null; @@ -175,11 +179,18 @@ public RewriteDataFiles.Result execute() { Stream groupStream = toGroupStream(ctx, fileGroupsByPartition); - if (partialProgressEnabled) { - return doExecuteWithPartialProgress(ctx, groupStream, commitManager(startingSnapshotId)); - } else { - return doExecute(ctx, groupStream, commitManager(startingSnapshotId)); + Builder resultBuilder = + partialProgressEnabled + ? doExecuteWithPartialProgress(ctx, groupStream, commitManager(startingSnapshotId)) + : doExecute(ctx, groupStream, commitManager(startingSnapshotId)); + + if (removeDanglingDeletes) { + RemoveDanglingDeletesSparkAction action = + new RemoveDanglingDeletesSparkAction(spark(), table); + int removedCount = Iterables.size(action.execute().removedDeleteFiles()); + resultBuilder.removedDeleteFilesCount(removedCount); } + return resultBuilder.build(); } StructLikeMap>> planFileGroups(long startingSnapshotId) { @@ -264,7 +275,7 @@ RewriteDataFilesCommitManager commitManager(long startingSnapshotId) { table, startingSnapshotId, useStartingSequenceNumber, commitSummary()); } - private Result doExecute( + private Builder doExecute( RewriteExecutionContext ctx, Stream groupStream, RewriteDataFilesCommitManager commitManager) { @@ -326,10 +337,10 @@ private Result doExecute( List rewriteResults = rewrittenGroups.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); - return ImmutableRewriteDataFiles.Result.builder().rewriteResults(rewriteResults).build(); + return ImmutableRewriteDataFiles.Result.builder().rewriteResults(rewriteResults); } - private Result doExecuteWithPartialProgress( + private Builder doExecuteWithPartialProgress( RewriteExecutionContext ctx, Stream groupStream, RewriteDataFilesCommitManager commitManager) { @@ -386,8 +397,7 @@ private Result doExecuteWithPartialProgress( return ImmutableRewriteDataFiles.Result.builder() .rewriteResults(toRewriteResults(commitService.results())) - .rewriteFailures(rewriteFailures) - .build(); + .rewriteFailures(rewriteFailures); } Stream toGroupStream( @@ -456,6 +466,10 @@ void validateAndInitOptions() { PropertyUtil.propertyAsBoolean( options(), USE_STARTING_SEQUENCE_NUMBER, USE_STARTING_SEQUENCE_NUMBER_DEFAULT); + removeDanglingDeletes = + PropertyUtil.propertyAsBoolean( + options(), REMOVE_DANGLING_DELETES, REMOVE_DANGLING_DELETES_DEFAULT); + rewriteJobOrder = RewriteJobOrder.fromName( PropertyUtil.propertyAsString(options(), REWRITE_JOB_ORDER, REWRITE_JOB_ORDER_DEFAULT)); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index f845386d30c4..ba9fa2e7b4db 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -21,6 +21,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -102,4 +103,9 @@ public RewritePositionDeleteFilesSparkAction rewritePositionDeletes(Table table) public ComputeTableStats computeTableStats(Table table) { return new ComputeTableStatsSparkAction(spark, table); } + + @Override + public RemoveDanglingDeleteFiles removeDanglingDeleteFiles(Table table) { + return new RemoveDanglingDeletesSparkAction(spark, table); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java new file mode 100644 index 000000000000..e15b2fb2174a --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -0,0 +1,447 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.nio.file.Path; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Encoders; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import scala.Tuple2; + +public class TestRemoveDanglingDeleteAction extends TestBase { + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.StringType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + + static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_A2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + + static final DataFile FILE_UNPARTITIONED = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-unpartitioned.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_POS_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_EQ_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + @TempDir private Path temp; + + private String tableLocation = null; + private Table table; + + @BeforeEach + public void before() throws Exception { + File tableDir = temp.resolve("junit").toFile(); + this.tableLocation = tableDir.toURI().toString(); + } + + @AfterEach + public void after() { + TABLES.dropTable(tableLocation); + } + + private void setupPartitionedTable() { + this.table = + TABLES.create( + SCHEMA, SPEC, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), tableLocation); + } + + private void setupUnpartitionedTable() { + this.table = + TABLES.create( + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + } + + @Test + public void testPartitionedDeletesWithLesserSeqNo() { + setupPartitionedTable(); + + // Add Data Files + table.newAppend().appendFile(FILE_B).appendFile(FILE_C).appendFile(FILE_D).commit(); + + // Add Delete Files + table + .newRowDelta() + .addDeletes(FILE_A_POS_DELETES) + .addDeletes(FILE_A2_POS_DELETES) + .addDeletes(FILE_B_POS_DELETES) + .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + + // Add More Data Files + table + .newAppend() + .appendFile(FILE_A2) + .appendFile(FILE_B2) + .appendFile(FILE_C2) + .appendFile(FILE_D2) + .commit(); + + List> actual = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(3L, FILE_A2.path().toString()), + Tuple2.apply(3L, FILE_B2.path().toString()), + Tuple2.apply(3L, FILE_C2.path().toString()), + Tuple2.apply(3L, FILE_D2.path().toString())); + assertThat(actual).isEqualTo(expected); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + + // All Delete files of the FILE A partition should be removed + // because there are no data files in partition with a lesser sequence number + + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::path) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected 4 delete files removed") + .hasSize(4) + .containsExactlyInAnyOrder( + FILE_A_POS_DELETES.path(), + FILE_A2_POS_DELETES.path(), + FILE_A_EQ_DELETES.path(), + FILE_A2_EQ_DELETES.path()); + + List> actualAfter = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .filter("status < 2") // live files + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(3L, FILE_A2.path().toString()), + Tuple2.apply(3L, FILE_B2.path().toString()), + Tuple2.apply(3L, FILE_C2.path().toString()), + Tuple2.apply(3L, FILE_D2.path().toString())); + assertThat(actualAfter).isEqualTo(expectedAfter); + } + + @Test + public void testPartitionedDeletesWithEqSeqNo() { + setupPartitionedTable(); + + // Add Data Files + table.newAppend().appendFile(FILE_A).appendFile(FILE_C).appendFile(FILE_D).commit(); + + // Add Data Files with EQ and POS deletes + table + .newRowDelta() + .addRows(FILE_A2) + .addRows(FILE_B2) + .addRows(FILE_C2) + .addRows(FILE_D2) + .addDeletes(FILE_A_POS_DELETES) + .addDeletes(FILE_A2_POS_DELETES) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(FILE_B_POS_DELETES) + .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + + List> actual = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_C2.path().toString()), + Tuple2.apply(2L, FILE_D2.path().toString())); + assertThat(actual).isEqualTo(expected); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + + // Eq Delete files of the FILE B partition should be removed + // because there are no data files in partition with a lesser sequence number + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::path) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected two delete files removed") + .hasSize(2) + .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.path(), FILE_B2_EQ_DELETES.path()); + + List> actualAfter = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .filter("status < 2") // live files + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_C2.path().toString()), + Tuple2.apply(2L, FILE_D2.path().toString())); + assertThat(actualAfter).isEqualTo(expectedAfter); + } + + @Test + public void testUnpartitionedTable() { + setupUnpartitionedTable(); + + table + .newRowDelta() + .addDeletes(FILE_UNPARTITIONED_POS_DELETE) + .addDeletes(FILE_UNPARTITIONED_EQ_DELETE) + .commit(); + table.newAppend().appendFile(FILE_UNPARTITIONED).commit(); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + assertThat(result.removedDeleteFiles()).as("No-op for unpartitioned tables").isEmpty(); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index b67ee87c7d3e..2de83f8b355c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -24,6 +24,7 @@ import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.min; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; @@ -56,6 +57,7 @@ import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RewriteJobOrder; import org.apache.iceberg.RowDelta; @@ -73,7 +75,9 @@ import org.apache.iceberg.actions.SizeBasedDataRewriter; import org.apache.iceberg.actions.SizeBasedFileRewriter; import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteWriter; import org.apache.iceberg.encryption.EncryptedFiles; @@ -86,6 +90,7 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; 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.ImmutableSet; @@ -105,9 +110,11 @@ import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.util.ArrayUtil; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.internal.SQLConf; import org.junit.jupiter.api.BeforeAll; @@ -128,6 +135,8 @@ public class TestRewriteDataFilesAction extends TestBase { optional(2, "c2", Types.StringType.get()), optional(3, "c3", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + @TempDir private Path temp; private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); @@ -336,6 +345,125 @@ public void testBinPackWithDeletes() { assertThat(actualRecords).as("7 rows are removed").hasSize(total - 7); } + @Test + public void testRemoveDangledEqualityDeletesPartitionEvolution() { + Table table = + TABLES.create( + SCHEMA, + SPEC, + Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + + // data seq = 1, write 4 files in 2 partitions + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(0, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(0, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + table.refresh(); + shouldHaveFiles(table, 4); + + // data seq = 2 & 3, write 2 equality deletes in both partitions + writeEqDeleteRecord(table, "c1", 1, "c3", "AAAA"); + writeEqDeleteRecord(table, "c1", 2, "c3", "CCCC"); + table.refresh(); + Set existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes) + .as("Only one equality delete c1=1 is used in query planning") + .hasSize(1); + + // partition evolution + table.refresh(); + table.updateSpec().addField(Expressions.ref("c3")).commit(); + + // data seq = 4, write 2 new data files in both partitions for evolved spec + List records3 = + Lists.newArrayList( + new ThreeColumnRecord(1, "A", "CCCC"), new ThreeColumnRecord(2, "D", "DDDD")); + writeRecords(records3); + + List originalData = currentData(); + + RewriteDataFiles.Result result = + basicRewrite(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .filter(Expressions.equal("c1", 1)) + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + + existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes).as("Shall pruned dangling deletes after rewrite").hasSize(0); + + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should compact 3 data files into 2 and remove both dangled equality delete file") + .containsExactly(2, 3, 2); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 5); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 7); + shouldHaveFiles(table, 5); + } + + @Test + public void testRemoveDangledPositionDeletesPartitionEvolution() { + Table table = + TABLES.create( + SCHEMA, + SPEC, + Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + + // data seq = 1, write 4 files in 2 partitions + writeRecords(2, 2, 2); + List dataFilesBefore = TestHelpers.dataFiles(table, null); + shouldHaveFiles(table, 4); + + // data seq = 2, write 1 position deletes in c1=1 + table + .newRowDelta() + .addDeletes(writePosDeletesToFile(table, dataFilesBefore.get(3), 1).get(0)) + .commit(); + + // partition evolution + table.updateSpec().addField(Expressions.ref("c3")).commit(); + + // data seq = 3, write 1 new data files in c1=1 for evolved spec + writeRecords(1, 1, 1); + shouldHaveFiles(table, 5); + List expectedRecords = currentData(); + + Result result = + actions() + .rewriteDataFiles(table) + .filter(Expressions.equal("c1", 1)) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should rewrite 2 data files into 1 and remove 1 dangled position delete file") + .containsExactly(1, 2, 1); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 3); + + shouldHaveSnapshots(table, 5); + assertThat(table.currentSnapshot().summary().get("total-position-deletes")).isEqualTo("0"); + assertEquals("Rows must match", expectedRecords, currentData()); + } + @Test public void testBinPackWithDeleteAllData() { Map options = Maps.newHashMap(); @@ -1697,6 +1825,21 @@ protected void shouldHaveFiles(Table table, int numExpected) { assertThat(numFiles).as("Did not have the expected number of files").isEqualTo(numExpected); } + protected long shouldHaveMinSequenceNumberInPartition( + Table table, String partitionFilter, long expected) { + long actual = + SparkTableUtil.loadMetadataTable(spark, table, MetadataTableType.ENTRIES) + .filter("status != 2") + .filter(partitionFilter) + .select("sequence_number") + .agg(min("sequence_number")) + .as(Encoders.LONG()) + .collectAsList() + .get(0); + assertThat(actual).as("Did not have the expected min sequence number").isEqualTo(expected); + return actual; + } + protected void shouldHaveSnapshots(Table table, int expectedSnapshots) { table.refresh(); int actualSnapshots = Iterables.size(table.snapshots()); @@ -1893,6 +2036,11 @@ protected int averageFileSize(Table table) { .getAsDouble(); } + private void writeRecords(List records) { + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + writeDF(df); + } + private void writeRecords(int files, int numRecords) { writeRecords(files, numRecords, 0); } @@ -1946,7 +2094,10 @@ private List writePosDeletes( table .io() .newOutputFile( - table.locationProvider().newDataLocation(UUID.randomUUID().toString())); + table + .locationProvider() + .newDataLocation( + FileFormat.PARQUET.addExtension(UUID.randomUUID().toString()))); EncryptedOutputFile encryptedOutputFile = EncryptedFiles.encryptedOutput(outputFile, EncryptionKeyMetadata.EMPTY); @@ -1972,6 +2123,68 @@ private List writePosDeletes( return results; } + private void writeEqDeleteRecord( + Table table, String partCol, Object partVal, String delCol, Object delVal) { + List equalityFieldIds = Lists.newArrayList(table.schema().findField(delCol).fieldId()); + Schema eqDeleteRowSchema = table.schema().select(delCol); + Record partitionRecord = + GenericRecord.create(table.schema().select(partCol)) + .copy(ImmutableMap.of(partCol, partVal)); + Record record = GenericRecord.create(eqDeleteRowSchema).copy(ImmutableMap.of(delCol, delVal)); + writeEqDeleteRecord(table, equalityFieldIds, partitionRecord, eqDeleteRowSchema, record); + } + + private void writeEqDeleteRecord( + Table table, + List equalityFieldIds, + Record partitionRecord, + Schema eqDeleteRowSchema, + Record deleteRecord) { + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PARQUET).build(); + GenericAppenderFactory appenderFactory = + new GenericAppenderFactory( + table.schema(), + table.spec(), + ArrayUtil.toIntArray(equalityFieldIds), + eqDeleteRowSchema, + null); + + EncryptedOutputFile file = + createEncryptedOutputFile(createPartitionKey(table, partitionRecord), fileFactory); + + EqualityDeleteWriter eqDeleteWriter = + appenderFactory.newEqDeleteWriter( + file, FileFormat.PARQUET, createPartitionKey(table, partitionRecord)); + + try (EqualityDeleteWriter clsEqDeleteWriter = eqDeleteWriter) { + clsEqDeleteWriter.write(deleteRecord); + } catch (Exception e) { + throw new RuntimeException(e); + } + table.newRowDelta().addDeletes(eqDeleteWriter.toDeleteFile()).commit(); + } + + private PartitionKey createPartitionKey(Table table, Record record) { + if (table.spec().isUnpartitioned()) { + return null; + } + + PartitionKey partitionKey = new PartitionKey(table.spec(), table.schema()); + partitionKey.partition(record); + + return partitionKey; + } + + private EncryptedOutputFile createEncryptedOutputFile( + PartitionKey partition, OutputFileFactory fileFactory) { + if (partition == null) { + return fileFactory.newOutputFile(); + } else { + return fileFactory.newOutputFile(partition); + } + } + private SparkActions actions() { return SparkActions.get(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 37b6cd86fb92..8547f9753f5e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -862,6 +862,7 @@ private void writePosDeletesForFiles( files.stream().collect(Collectors.groupingBy(ContentFile::partition)); List deleteFiles = Lists.newArrayListWithCapacity(deleteFilesPerPartition * filesByPartition.size()); + String suffix = String.format(".%s", FileFormat.PARQUET.name().toLowerCase()); for (Map.Entry> filesByPartitionEntry : filesByPartition.entrySet()) { @@ -886,7 +887,7 @@ private void writePosDeletesForFiles( if (counter == deleteFileSize) { // Dump to file and reset variables OutputFile output = - Files.localOutput(File.createTempFile("junit", null, temp.toFile())); + Files.localOutput(File.createTempFile("junit", suffix, temp.toFile())); deleteFiles.add(FileHelpers.writeDeleteFile(table, output, partition, deletes).first()); counter = 0; deletes.clear(); From 97946088a8977e47ad996220186d4e137edc3d28 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 23 Oct 2024 17:32:30 +0200 Subject: [PATCH 030/313] Spark: Randomize view/function names in testing (#11381) --- .../iceberg/spark/extensions/TestViews.java | 169 +++++++++--------- 1 file changed, 87 insertions(+), 82 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index d521cc72cb31..61eafc5f4e78 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; +import java.util.Locale; import java.util.Random; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -91,7 +92,7 @@ public static Object[][] parameters() { @TestTemplate public void readFromView() throws NoSuchTableException { insertRows(10); - String viewName = "simpleView"; + String viewName = viewName("simpleView"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -117,7 +118,7 @@ public void readFromView() throws NoSuchTableException { @TestTemplate public void readFromTrinoView() throws NoSuchTableException { insertRows(10); - String viewName = "trinoView"; + String viewName = viewName("trinoView"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -142,8 +143,8 @@ public void readFromTrinoView() throws NoSuchTableException { @TestTemplate public void readFromMultipleViews() throws NoSuchTableException { insertRows(6); - String viewName = "firstView"; - String secondView = "secondView"; + String viewName = viewName("firstView"); + String secondView = viewName("secondView"); String viewSQL = String.format("SELECT id FROM %s WHERE id <= 3", tableName); String secondViewSQL = String.format("SELECT id FROM %s WHERE id > 3", tableName); @@ -175,7 +176,7 @@ public void readFromMultipleViews() throws NoSuchTableException { @TestTemplate public void readFromViewUsingNonExistingTable() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithNonExistingTable"; + String viewName = viewName("viewWithNonExistingTable"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = new Schema(Types.NestedField.required(1, "id", Types.LongType.get())); @@ -199,7 +200,7 @@ public void readFromViewUsingNonExistingTable() throws NoSuchTableException { @TestTemplate public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithNonExistingColumn"; + String viewName = viewName("viewWithNonExistingColumn"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = new Schema(Types.NestedField.required(1, "non_existing", Types.LongType.get())); @@ -221,7 +222,7 @@ public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableExceptio @TestTemplate public void readFromViewUsingInvalidSQL() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithInvalidSQL"; + String viewName = viewName("viewWithInvalidSQL"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -243,7 +244,7 @@ public void readFromViewUsingInvalidSQL() throws NoSuchTableException { @TestTemplate public void readFromViewWithStaleSchema() throws NoSuchTableException { insertRows(10); - String viewName = "staleView"; + String viewName = viewName("staleView"); String sql = String.format("SELECT id, data FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -269,7 +270,7 @@ public void readFromViewWithStaleSchema() throws NoSuchTableException { @TestTemplate public void readFromViewHiddenByTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewHiddenByTempView"; + String viewName = viewName("viewHiddenByTempView"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -296,7 +297,7 @@ public void readFromViewHiddenByTempView() throws NoSuchTableException { @TestTemplate public void readFromViewWithGlobalTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithGlobalTempView"; + String viewName = viewName("viewWithGlobalTempView"); String sql = String.format("SELECT id FROM %s WHERE id > 5", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -326,8 +327,8 @@ public void readFromViewWithGlobalTempView() throws NoSuchTableException { @TestTemplate public void readFromViewReferencingAnotherView() throws NoSuchTableException { insertRows(10); - String firstView = "viewBeingReferencedInAnotherView"; - String viewReferencingOtherView = "viewReferencingOtherView"; + String firstView = viewName("viewBeingReferencedInAnotherView"); + String viewReferencingOtherView = viewName("viewReferencingOtherView"); String firstSQL = String.format("SELECT id FROM %s WHERE id <= 5", tableName); String secondSQL = String.format("SELECT id FROM %s WHERE id > 4", firstView); @@ -357,8 +358,8 @@ public void readFromViewReferencingAnotherView() throws NoSuchTableException { @TestTemplate public void readFromViewReferencingTempView() throws NoSuchTableException { insertRows(10); - String tempView = "tempViewBeingReferencedInAnotherView"; - String viewReferencingTempView = "viewReferencingTempView"; + String tempView = viewName("tempViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTempView"); String sql = String.format("SELECT id FROM %s", tempView); ViewCatalog viewCatalog = viewCatalog(); @@ -393,8 +394,8 @@ public void readFromViewReferencingTempView() throws NoSuchTableException { @TestTemplate public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTableException { insertRows(10); - String innerViewName = "inner_view"; - String outerViewName = "outer_view"; + String innerViewName = viewName("inner_view"); + String outerViewName = viewName("outer_view"); String innerViewSQL = String.format("SELECT * FROM %s WHERE id > 5", tableName); String outerViewSQL = String.format("SELECT id FROM %s", innerViewName); @@ -441,8 +442,8 @@ public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTa @TestTemplate public void readFromViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); - String globalTempView = "globalTempViewBeingReferenced"; - String viewReferencingTempView = "viewReferencingGlobalTempView"; + String globalTempView = viewName("globalTempViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTempView"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -480,7 +481,7 @@ public void readFromViewReferencingGlobalTempView() throws NoSuchTableException public void readFromViewReferencingTempFunction() throws NoSuchTableException { insertRows(10); String viewName = viewName("viewReferencingTempFunction"); - String functionName = "test_avg"; + String functionName = viewName("test_avg"); String sql = String.format("SELECT %s(id) FROM %s", functionName, tableName); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -511,7 +512,7 @@ public void readFromViewReferencingTempFunction() throws NoSuchTableException { @TestTemplate public void readFromViewWithCTE() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithCTE"; + String viewName = viewName("viewWithCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -533,7 +534,7 @@ public void readFromViewWithCTE() throws NoSuchTableException { @TestTemplate public void rewriteFunctionIdentifier() { - String viewName = "rewriteFunctionIdentifier"; + String viewName = viewName("rewriteFunctionIdentifier"); String sql = "SELECT iceberg_version() AS version"; assertThatThrownBy(() -> sql(sql)) @@ -559,7 +560,7 @@ public void rewriteFunctionIdentifier() { @TestTemplate public void builtinFunctionIdentifierNotRewritten() { - String viewName = "builtinFunctionIdentifierNotRewritten"; + String viewName = viewName("builtinFunctionIdentifierNotRewritten"); String sql = "SELECT trim(' abc ') AS result"; ViewCatalog viewCatalog = viewCatalog(); @@ -578,7 +579,7 @@ public void builtinFunctionIdentifierNotRewritten() { @TestTemplate public void rewriteFunctionIdentifierWithNamespace() { - String viewName = "rewriteFunctionIdentifierWithNamespace"; + String viewName = viewName("rewriteFunctionIdentifierWithNamespace"); String sql = "SELECT system.bucket(100, 'a') AS bucket_result, 'a' AS value"; ViewCatalog viewCatalog = viewCatalog(); @@ -605,7 +606,7 @@ public void rewriteFunctionIdentifierWithNamespace() { @TestTemplate public void fullFunctionIdentifier() { - String viewName = "fullFunctionIdentifier"; + String viewName = viewName("fullFunctionIdentifier"); String sql = String.format( "SELECT %s.system.bucket(100, 'a') AS bucket_result, 'a' AS value", catalogName); @@ -629,7 +630,7 @@ public void fullFunctionIdentifier() { @TestTemplate public void fullFunctionIdentifierNotRewrittenLoadFailure() { - String viewName = "fullFunctionIdentifierNotRewrittenLoadFailure"; + String viewName = viewName("fullFunctionIdentifierNotRewrittenLoadFailure"); String sql = "SELECT spark_catalog.system.bucket(100, 'a') AS bucket_result, 'a' AS value"; // avoid namespace failures @@ -821,7 +822,7 @@ public void renameViewTargetAlreadyExistsAsTable() { @TestTemplate public void dropView() { - String viewName = "viewToBeDropped"; + String viewName = viewName("viewToBeDropped"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -850,7 +851,7 @@ public void dropNonExistingView() { @TestTemplate public void dropViewIfExists() { - String viewName = "viewToBeDropped"; + String viewName = viewName("viewToBeDropped"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -875,7 +876,7 @@ public void dropViewIfExists() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @TestTemplate public void dropGlobalTempView() { - String globalTempView = "globalViewToBeDropped"; + String globalTempView = viewName("globalViewToBeDropped"); sql("CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s", globalTempView, tableName); assertThat(v1SessionCatalog().getGlobalTempView(globalTempView).isDefined()).isTrue(); @@ -886,7 +887,7 @@ public void dropGlobalTempView() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @TestTemplate public void dropTempView() { - String tempView = "tempViewToBeDropped"; + String tempView = viewName("tempViewToBeDropped"); sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s", tempView, tableName); assertThat(v1SessionCatalog().getTempView(tempView).isDefined()).isTrue(); @@ -897,7 +898,7 @@ public void dropTempView() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @TestTemplate public void dropV1View() { - String v1View = "v1ViewToBeDropped"; + String v1View = viewName("v1ViewToBeDropped"); sql("USE spark_catalog"); sql("CREATE NAMESPACE IF NOT EXISTS %s", NAMESPACE); sql("CREATE TABLE %s (id INT, data STRING)", tableName); @@ -928,7 +929,7 @@ private String viewName(String viewName) { @TestTemplate public void createViewIfNotExists() { - String viewName = "viewThatAlreadyExists"; + String viewName = viewName("viewThatAlreadyExists"); sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName); assertThatThrownBy(() -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName)) @@ -969,8 +970,8 @@ public void createViewWithInvalidSQL() { @TestTemplate public void createViewReferencingTempView() throws NoSuchTableException { insertRows(10); - String tempView = "temporaryViewBeingReferencedInAnotherView"; - String viewReferencingTempView = "viewReferencingTemporaryView"; + String tempView = viewName("temporaryViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTemporaryView"); sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", tempView, tableName); @@ -988,8 +989,8 @@ public void createViewReferencingTempView() throws NoSuchTableException { @TestTemplate public void createViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); - String globalTempView = "globalTemporaryViewBeingReferenced"; - String viewReferencingTempView = "viewReferencingGlobalTemporaryView"; + String globalTempView = viewName("globalTemporaryViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTemporaryView"); sql( "CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", @@ -1012,7 +1013,7 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { @TestTemplate public void createViewReferencingTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); - String functionName = "test_avg_func"; + String functionName = viewName("test_avg_func"); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -1031,7 +1032,7 @@ public void createViewReferencingTempFunction() { @TestTemplate public void createViewReferencingQualifiedTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); - String functionName = "test_avg_func_qualified"; + String functionName = viewName("test_avg_func_qualified"); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -1069,7 +1070,7 @@ public void createViewUsingNonExistingTable() { @TestTemplate public void createViewWithMismatchedColumnCounts() { - String viewName = "viewWithMismatchedColumnCounts"; + String viewName = viewName("viewWithMismatchedColumnCounts"); assertThatThrownBy( () -> sql("CREATE VIEW %s (id, data) AS SELECT id FROM %s", viewName, tableName)) @@ -1093,7 +1094,7 @@ public void createViewWithMismatchedColumnCounts() { @TestTemplate public void createViewWithColumnAliases() throws NoSuchTableException { insertRows(6); - String viewName = "viewWithColumnAliases"; + String viewName = viewName("viewWithColumnAliases"); sql( "CREATE VIEW %s (new_id COMMENT 'ID', new_data COMMENT 'DATA') AS SELECT id, data FROM %s WHERE id <= 3", @@ -1140,7 +1141,7 @@ public void createViewWithDuplicateColumnNames() { @TestTemplate public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableException { insertRows(3); - String viewName = "viewWithDuplicateQueryColumnNames"; + String viewName = viewName("viewWithDuplicateQueryColumnNames"); String sql = String.format("SELECT id, id FROM %s WHERE id <= 3", tableName); // not specifying column aliases in the view should fail @@ -1158,7 +1159,7 @@ public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableExceptio @TestTemplate public void createViewWithCTE() throws NoSuchTableException { insertRows(10); - String viewName = "simpleViewWithCTE"; + String viewName = viewName("simpleViewWithCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -1173,8 +1174,8 @@ public void createViewWithCTE() throws NoSuchTableException { @TestTemplate public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithConflictingNamesForCTEAndTempView"; - String cteName = "cteName"; + String viewName = viewName("viewWithConflictingNamesForCTEAndTempView"); + String cteName = viewName("cteName"); String sql = String.format( "WITH %s AS (SELECT max(id) as max FROM %s) " @@ -1191,8 +1192,8 @@ public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTable @TestTemplate public void createViewWithCTEReferencingTempView() { - String viewName = "viewWithCTEReferencingTempView"; - String tempViewInCTE = "tempViewInCTE"; + String viewName = viewName("viewWithCTEReferencingTempView"); + String tempViewInCTE = viewName("tempViewInCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -1211,8 +1212,8 @@ public void createViewWithCTEReferencingTempView() { @TestTemplate public void createViewWithCTEReferencingTempFunction() { - String viewName = "viewWithCTEReferencingTempFunction"; - String functionName = "avg_function_in_cte"; + String viewName = viewName("viewWithCTEReferencingTempFunction"); + String functionName = viewName("avg_function_in_cte"); String sql = String.format( "WITH avg_data AS (SELECT %s(id) as avg FROM %s) " @@ -1245,8 +1246,8 @@ public void createViewWithNonExistingQueryColumn() { @TestTemplate public void createViewWithSubqueryExpressionUsingTempView() { - String viewName = "viewWithSubqueryExpression"; - String tempView = "simpleTempView"; + String viewName = viewName("viewWithSubqueryExpression"); + String tempView = viewName("simpleTempView"); String sql = String.format("SELECT * FROM %s WHERE id = (SELECT id FROM %s)", tableName, tempView); @@ -1262,8 +1263,8 @@ public void createViewWithSubqueryExpressionUsingTempView() { @TestTemplate public void createViewWithSubqueryExpressionUsingGlobalTempView() { - String viewName = "simpleViewWithSubqueryExpression"; - String globalTempView = "simpleGlobalTempView"; + String viewName = viewName("simpleViewWithSubqueryExpression"); + String globalTempView = viewName("simpleGlobalTempView"); String sql = String.format( "SELECT * FROM %s WHERE id = (SELECT id FROM global_temp.%s)", @@ -1284,7 +1285,7 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { @TestTemplate public void createViewWithSubqueryExpressionUsingTempFunction() { String viewName = viewName("viewWithSubqueryExpression"); - String functionName = "avg_function_in_subquery"; + String functionName = viewName("avg_function_in_subquery"); String sql = String.format( "SELECT * FROM %s WHERE id < (SELECT %s(id) FROM %s)", @@ -1354,7 +1355,7 @@ public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSu @TestTemplate public void describeView() { - String viewName = "describeView"; + String viewName = viewName("describeView"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); assertThat(sql("DESCRIBE %s", viewName)) @@ -1363,7 +1364,7 @@ public void describeView() { @TestTemplate public void describeExtendedView() { - String viewName = "describeExtendedView"; + String viewName = viewName("describeExtendedView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql( @@ -1388,7 +1389,7 @@ public void describeExtendedView() { @TestTemplate public void showViewProperties() { - String viewName = "showViewProps"; + String viewName = viewName("showViewProps"); sql( "CREATE VIEW %s TBLPROPERTIES ('key1'='val1', 'key2'='val2') AS SELECT id, data FROM %s WHERE id <= 3", @@ -1399,7 +1400,7 @@ public void showViewProperties() { @TestTemplate public void showViewPropertiesByKey() { - String viewName = "showViewPropsByKey"; + String viewName = viewName("showViewPropsByKey"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); assertThat(sql("SHOW TBLPROPERTIES %s", viewName)).contains(row("provider", "iceberg")); @@ -1420,39 +1421,43 @@ public void showViewPropertiesByKey() { public void showViews() throws NoSuchTableException { insertRows(6); String sql = String.format("SELECT * from %s", tableName); - sql("CREATE VIEW v1 AS %s", sql); - sql("CREATE VIEW prefixV2 AS %s", sql); - sql("CREATE VIEW prefixV3 AS %s", sql); - sql("CREATE GLOBAL TEMPORARY VIEW globalViewForListing AS %s", sql); - sql("CREATE TEMPORARY VIEW tempViewForListing AS %s", sql); + String v1 = viewName("v1"); + String prefixV2 = viewName("prefixV2"); + String prefixV3 = viewName("prefixV3"); + String globalViewForListing = viewName("globalViewForListing"); + String tempViewForListing = viewName("tempViewForListing"); + sql("CREATE VIEW %s AS %s", v1, sql); + sql("CREATE VIEW %s AS %s", prefixV2, sql); + sql("CREATE VIEW %s AS %s", prefixV3, sql); + sql("CREATE GLOBAL TEMPORARY VIEW %s AS %s", globalViewForListing, sql); + sql("CREATE TEMPORARY VIEW %s AS %s", tempViewForListing, sql); // spark stores temp views case-insensitive by default - Object[] tempView = row("", "tempviewforlisting", true); + Object[] tempView = row("", tempViewForListing.toLowerCase(Locale.ROOT), true); assertThat(sql("SHOW VIEWS")) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS IN %s", catalogName)) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, NAMESPACE)) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS LIKE 'pref*'")) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false)); + row(NAMESPACE.toString(), prefixV2, false), row(NAMESPACE.toString(), prefixV3, false)); assertThat(sql("SHOW VIEWS LIKE 'non-existing'")).isEmpty(); @@ -1461,7 +1466,7 @@ public void showViews() throws NoSuchTableException { assertThat(sql("SHOW VIEWS IN global_temp")) .contains( // spark stores temp views case-insensitive by default - row("global_temp", "globalviewforlisting", true), tempView); + row("global_temp", globalViewForListing.toLowerCase(Locale.ROOT), true), tempView); sql("USE spark_catalog"); assertThat(sql("SHOW VIEWS")).contains(tempView); @@ -1502,7 +1507,7 @@ public void showViewsWithCurrentNamespace() { @TestTemplate public void showCreateSimpleView() { - String viewName = "showCreateSimpleView"; + String viewName = viewName("showCreateSimpleView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql("CREATE VIEW %s AS %s", viewName, sql); @@ -1523,7 +1528,7 @@ public void showCreateSimpleView() { @TestTemplate public void showCreateComplexView() { - String viewName = "showCreateComplexView"; + String viewName = viewName("showCreateComplexView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql( @@ -1550,7 +1555,7 @@ public void showCreateComplexView() { @TestTemplate public void alterViewSetProperties() { - String viewName = "viewWithSetProperties"; + String viewName = viewName("viewWithSetProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1572,7 +1577,7 @@ public void alterViewSetProperties() { @TestTemplate public void alterViewSetReservedProperties() { - String viewName = "viewWithSetReservedProperties"; + String viewName = viewName("viewWithSetReservedProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1603,7 +1608,7 @@ public void alterViewSetReservedProperties() { @TestTemplate public void alterViewUnsetProperties() { - String viewName = "viewWithUnsetProperties"; + String viewName = viewName("viewWithUnsetProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -1624,7 +1629,7 @@ public void alterViewUnsetProperties() { @TestTemplate public void alterViewUnsetUnknownProperty() { - String viewName = "viewWithUnsetUnknownProp"; + String viewName = viewName("viewWithUnsetUnknownProp"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); assertThatThrownBy(() -> sql("ALTER VIEW %s UNSET TBLPROPERTIES ('unknown-key')", viewName)) @@ -1638,7 +1643,7 @@ public void alterViewUnsetUnknownProperty() { @TestTemplate public void alterViewUnsetReservedProperties() { - String viewName = "viewWithUnsetReservedProperties"; + String viewName = viewName("viewWithUnsetReservedProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1721,7 +1726,7 @@ public void createOrReplaceViewWithColumnAliases() throws NoSuchTableException { @TestTemplate public void alterViewIsNotSupported() throws NoSuchTableException { insertRows(6); - String viewName = "alteredView"; + String viewName = viewName("alteredView"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); From 60181f9a763fb3912f071ae0a3dcdaeff6ddb477 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 23 Oct 2024 19:16:39 +0200 Subject: [PATCH 031/313] Spark 3.4: Randomize view/function names in testing (#11382) --- .../iceberg/spark/extensions/TestViews.java | 169 +++++++++--------- 1 file changed, 87 insertions(+), 82 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index 65b8669c35a0..1380711ed7b7 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Random; import java.util.stream.Collectors; @@ -92,7 +93,7 @@ public TestViews(String catalog, String implementation, Map prop @Test public void readFromView() throws NoSuchTableException { insertRows(10); - String viewName = "simpleView"; + String viewName = viewName("simpleView"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -118,7 +119,7 @@ public void readFromView() throws NoSuchTableException { @Test public void readFromTrinoView() throws NoSuchTableException { insertRows(10); - String viewName = "trinoView"; + String viewName = viewName("trinoView"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -143,8 +144,8 @@ public void readFromTrinoView() throws NoSuchTableException { @Test public void readFromMultipleViews() throws NoSuchTableException { insertRows(6); - String viewName = "firstView"; - String secondView = "secondView"; + String viewName = viewName("firstView"); + String secondView = viewName("secondView"); String viewSQL = String.format("SELECT id FROM %s WHERE id <= 3", tableName); String secondViewSQL = String.format("SELECT id FROM %s WHERE id > 3", tableName); @@ -176,7 +177,7 @@ public void readFromMultipleViews() throws NoSuchTableException { @Test public void readFromViewUsingNonExistingTable() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithNonExistingTable"; + String viewName = viewName("viewWithNonExistingTable"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = new Schema(Types.NestedField.required(1, "id", Types.LongType.get())); @@ -200,7 +201,7 @@ public void readFromViewUsingNonExistingTable() throws NoSuchTableException { @Test public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithNonExistingColumn"; + String viewName = viewName("viewWithNonExistingColumn"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = new Schema(Types.NestedField.required(1, "non_existing", Types.LongType.get())); @@ -222,7 +223,7 @@ public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableExceptio @Test public void readFromViewUsingInvalidSQL() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithInvalidSQL"; + String viewName = viewName("viewWithInvalidSQL"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -244,7 +245,7 @@ public void readFromViewUsingInvalidSQL() throws NoSuchTableException { @Test public void readFromViewWithStaleSchema() throws NoSuchTableException { insertRows(10); - String viewName = "staleView"; + String viewName = viewName("staleView"); String sql = String.format("SELECT id, data FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -270,7 +271,7 @@ public void readFromViewWithStaleSchema() throws NoSuchTableException { @Test public void readFromViewHiddenByTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewHiddenByTempView"; + String viewName = viewName("viewHiddenByTempView"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -297,7 +298,7 @@ public void readFromViewHiddenByTempView() throws NoSuchTableException { @Test public void readFromViewWithGlobalTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithGlobalTempView"; + String viewName = viewName("viewWithGlobalTempView"); String sql = String.format("SELECT id FROM %s WHERE id > 5", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -327,8 +328,8 @@ public void readFromViewWithGlobalTempView() throws NoSuchTableException { @Test public void readFromViewReferencingAnotherView() throws NoSuchTableException { insertRows(10); - String firstView = "viewBeingReferencedInAnotherView"; - String viewReferencingOtherView = "viewReferencingOtherView"; + String firstView = viewName("viewBeingReferencedInAnotherView"); + String viewReferencingOtherView = viewName("viewReferencingOtherView"); String firstSQL = String.format("SELECT id FROM %s WHERE id <= 5", tableName); String secondSQL = String.format("SELECT id FROM %s WHERE id > 4", firstView); @@ -358,8 +359,8 @@ public void readFromViewReferencingAnotherView() throws NoSuchTableException { @Test public void readFromViewReferencingTempView() throws NoSuchTableException { insertRows(10); - String tempView = "tempViewBeingReferencedInAnotherView"; - String viewReferencingTempView = "viewReferencingTempView"; + String tempView = viewName("tempViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTempView"); String sql = String.format("SELECT id FROM %s", tempView); ViewCatalog viewCatalog = viewCatalog(); @@ -394,8 +395,8 @@ public void readFromViewReferencingTempView() throws NoSuchTableException { @Test public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTableException { insertRows(10); - String innerViewName = "inner_view"; - String outerViewName = "outer_view"; + String innerViewName = viewName("inner_view"); + String outerViewName = viewName("outer_view"); String innerViewSQL = String.format("SELECT * FROM %s WHERE id > 5", tableName); String outerViewSQL = String.format("SELECT id FROM %s", innerViewName); @@ -442,8 +443,8 @@ public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTa @Test public void readFromViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); - String globalTempView = "globalTempViewBeingReferenced"; - String viewReferencingTempView = "viewReferencingGlobalTempView"; + String globalTempView = viewName("globalTempViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTempView"); ViewCatalog viewCatalog = viewCatalog(); Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); @@ -481,7 +482,7 @@ public void readFromViewReferencingGlobalTempView() throws NoSuchTableException public void readFromViewReferencingTempFunction() throws NoSuchTableException { insertRows(10); String viewName = viewName("viewReferencingTempFunction"); - String functionName = "test_avg"; + String functionName = viewName("test_avg"); String sql = String.format("SELECT %s(id) FROM %s", functionName, tableName); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -512,7 +513,7 @@ public void readFromViewReferencingTempFunction() throws NoSuchTableException { @Test public void readFromViewWithCTE() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithCTE"; + String viewName = viewName("viewWithCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -534,7 +535,7 @@ public void readFromViewWithCTE() throws NoSuchTableException { @Test public void rewriteFunctionIdentifier() { - String viewName = "rewriteFunctionIdentifier"; + String viewName = viewName("rewriteFunctionIdentifier"); String sql = "SELECT iceberg_version() AS version"; assertThatThrownBy(() -> sql(sql)) @@ -560,7 +561,7 @@ public void rewriteFunctionIdentifier() { @Test public void builtinFunctionIdentifierNotRewritten() { - String viewName = "builtinFunctionIdentifierNotRewritten"; + String viewName = viewName("builtinFunctionIdentifierNotRewritten"); String sql = "SELECT trim(' abc ') AS result"; ViewCatalog viewCatalog = viewCatalog(); @@ -579,7 +580,7 @@ public void builtinFunctionIdentifierNotRewritten() { @Test public void rewriteFunctionIdentifierWithNamespace() { - String viewName = "rewriteFunctionIdentifierWithNamespace"; + String viewName = viewName("rewriteFunctionIdentifierWithNamespace"); String sql = "SELECT system.bucket(100, 'a') AS bucket_result, 'a' AS value"; ViewCatalog viewCatalog = viewCatalog(); @@ -606,7 +607,7 @@ public void rewriteFunctionIdentifierWithNamespace() { @Test public void fullFunctionIdentifier() { - String viewName = "fullFunctionIdentifier"; + String viewName = viewName("fullFunctionIdentifier"); String sql = String.format( "SELECT %s.system.bucket(100, 'a') AS bucket_result, 'a' AS value", catalogName); @@ -630,7 +631,7 @@ public void fullFunctionIdentifier() { @Test public void fullFunctionIdentifierNotRewrittenLoadFailure() { - String viewName = "fullFunctionIdentifierNotRewrittenLoadFailure"; + String viewName = viewName("fullFunctionIdentifierNotRewrittenLoadFailure"); String sql = "SELECT spark_catalog.system.bucket(100, 'a') AS bucket_result, 'a' AS value"; // avoid namespace failures @@ -822,7 +823,7 @@ public void renameViewTargetAlreadyExistsAsTable() { @Test public void dropView() { - String viewName = "viewToBeDropped"; + String viewName = viewName("viewToBeDropped"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -851,7 +852,7 @@ public void dropNonExistingView() { @Test public void dropViewIfExists() { - String viewName = "viewToBeDropped"; + String viewName = viewName("viewToBeDropped"); String sql = String.format("SELECT id FROM %s", tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -876,7 +877,7 @@ public void dropViewIfExists() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @Test public void dropGlobalTempView() { - String globalTempView = "globalViewToBeDropped"; + String globalTempView = viewName("globalViewToBeDropped"); sql("CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s", globalTempView, tableName); assertThat(v1SessionCatalog().getGlobalTempView(globalTempView).isDefined()).isTrue(); @@ -887,7 +888,7 @@ public void dropGlobalTempView() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @Test public void dropTempView() { - String tempView = "tempViewToBeDropped"; + String tempView = viewName("tempViewToBeDropped"); sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s", tempView, tableName); assertThat(v1SessionCatalog().getTempView(tempView).isDefined()).isTrue(); @@ -898,7 +899,7 @@ public void dropTempView() { /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ @Test public void dropV1View() { - String v1View = "v1ViewToBeDropped"; + String v1View = viewName("v1ViewToBeDropped"); sql("USE spark_catalog"); sql("CREATE NAMESPACE IF NOT EXISTS %s", NAMESPACE); sql("CREATE TABLE %s (id INT, data STRING)", tableName); @@ -929,7 +930,7 @@ private String viewName(String viewName) { @Test public void createViewIfNotExists() { - String viewName = "viewThatAlreadyExists"; + String viewName = viewName("viewThatAlreadyExists"); sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName); assertThatThrownBy(() -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName)) @@ -970,8 +971,8 @@ public void createViewWithInvalidSQL() { @Test public void createViewReferencingTempView() throws NoSuchTableException { insertRows(10); - String tempView = "temporaryViewBeingReferencedInAnotherView"; - String viewReferencingTempView = "viewReferencingTemporaryView"; + String tempView = viewName("temporaryViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTemporaryView"); sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", tempView, tableName); @@ -989,8 +990,8 @@ public void createViewReferencingTempView() throws NoSuchTableException { @Test public void createViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); - String globalTempView = "globalTemporaryViewBeingReferenced"; - String viewReferencingTempView = "viewReferencingGlobalTemporaryView"; + String globalTempView = viewName("globalTemporaryViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTemporaryView"); sql( "CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", @@ -1013,7 +1014,7 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { @Test public void createViewReferencingTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); - String functionName = "test_avg_func"; + String functionName = viewName("test_avg_func"); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -1032,7 +1033,7 @@ public void createViewReferencingTempFunction() { @Test public void createViewReferencingQualifiedTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); - String functionName = "test_avg_func_qualified"; + String functionName = viewName("test_avg_func_qualified"); sql( "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", @@ -1070,7 +1071,7 @@ public void createViewUsingNonExistingTable() { @Test public void createViewWithMismatchedColumnCounts() { - String viewName = "viewWithMismatchedColumnCounts"; + String viewName = viewName("viewWithMismatchedColumnCounts"); assertThatThrownBy( () -> sql("CREATE VIEW %s (id, data) AS SELECT id FROM %s", viewName, tableName)) @@ -1094,7 +1095,7 @@ public void createViewWithMismatchedColumnCounts() { @Test public void createViewWithColumnAliases() throws NoSuchTableException { insertRows(6); - String viewName = "viewWithColumnAliases"; + String viewName = viewName("viewWithColumnAliases"); sql( "CREATE VIEW %s (new_id COMMENT 'ID', new_data COMMENT 'DATA') AS SELECT id, data FROM %s WHERE id <= 3", @@ -1141,7 +1142,7 @@ public void createViewWithDuplicateColumnNames() { @Test public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableException { insertRows(3); - String viewName = "viewWithDuplicateQueryColumnNames"; + String viewName = viewName("viewWithDuplicateQueryColumnNames"); String sql = String.format("SELECT id, id FROM %s WHERE id <= 3", tableName); // not specifying column aliases in the view should fail @@ -1159,7 +1160,7 @@ public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableExceptio @Test public void createViewWithCTE() throws NoSuchTableException { insertRows(10); - String viewName = "simpleViewWithCTE"; + String viewName = viewName("simpleViewWithCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -1174,8 +1175,8 @@ public void createViewWithCTE() throws NoSuchTableException { @Test public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTableException { insertRows(10); - String viewName = "viewWithConflictingNamesForCTEAndTempView"; - String cteName = "cteName"; + String viewName = viewName("viewWithConflictingNamesForCTEAndTempView"); + String cteName = viewName("cteName"); String sql = String.format( "WITH %s AS (SELECT max(id) as max FROM %s) " @@ -1192,8 +1193,8 @@ public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTable @Test public void createViewWithCTEReferencingTempView() { - String viewName = "viewWithCTEReferencingTempView"; - String tempViewInCTE = "tempViewInCTE"; + String viewName = viewName("viewWithCTEReferencingTempView"); + String tempViewInCTE = viewName("tempViewInCTE"); String sql = String.format( "WITH max_by_data AS (SELECT max(id) as max FROM %s) " @@ -1212,8 +1213,8 @@ public void createViewWithCTEReferencingTempView() { @Test public void createViewWithCTEReferencingTempFunction() { - String viewName = "viewWithCTEReferencingTempFunction"; - String functionName = "avg_function_in_cte"; + String viewName = viewName("viewWithCTEReferencingTempFunction"); + String functionName = viewName("avg_function_in_cte"); String sql = String.format( "WITH avg_data AS (SELECT %s(id) as avg FROM %s) " @@ -1246,8 +1247,8 @@ public void createViewWithNonExistingQueryColumn() { @Test public void createViewWithSubqueryExpressionUsingTempView() { - String viewName = "viewWithSubqueryExpression"; - String tempView = "simpleTempView"; + String viewName = viewName("viewWithSubqueryExpression"); + String tempView = viewName("simpleTempView"); String sql = String.format("SELECT * FROM %s WHERE id = (SELECT id FROM %s)", tableName, tempView); @@ -1263,8 +1264,8 @@ public void createViewWithSubqueryExpressionUsingTempView() { @Test public void createViewWithSubqueryExpressionUsingGlobalTempView() { - String viewName = "simpleViewWithSubqueryExpression"; - String globalTempView = "simpleGlobalTempView"; + String viewName = viewName("simpleViewWithSubqueryExpression"); + String globalTempView = viewName("simpleGlobalTempView"); String sql = String.format( "SELECT * FROM %s WHERE id = (SELECT id FROM global_temp.%s)", @@ -1285,7 +1286,7 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { @Test public void createViewWithSubqueryExpressionUsingTempFunction() { String viewName = viewName("viewWithSubqueryExpression"); - String functionName = "avg_function_in_subquery"; + String functionName = viewName("avg_function_in_subquery"); String sql = String.format( "SELECT * FROM %s WHERE id < (SELECT %s(id) FROM %s)", @@ -1355,7 +1356,7 @@ public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSu @Test public void describeView() { - String viewName = "describeView"; + String viewName = viewName("describeView"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); assertThat(sql("DESCRIBE %s", viewName)) @@ -1364,7 +1365,7 @@ public void describeView() { @Test public void describeExtendedView() { - String viewName = "describeExtendedView"; + String viewName = viewName("describeExtendedView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql( @@ -1389,7 +1390,7 @@ public void describeExtendedView() { @Test public void showViewProperties() { - String viewName = "showViewProps"; + String viewName = viewName("showViewProps"); sql( "CREATE VIEW %s TBLPROPERTIES ('key1'='val1', 'key2'='val2') AS SELECT id, data FROM %s WHERE id <= 3", @@ -1400,7 +1401,7 @@ public void showViewProperties() { @Test public void showViewPropertiesByKey() { - String viewName = "showViewPropsByKey"; + String viewName = viewName("showViewPropsByKey"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); assertThat(sql("SHOW TBLPROPERTIES %s", viewName)).contains(row("provider", "iceberg")); @@ -1421,39 +1422,43 @@ public void showViewPropertiesByKey() { public void showViews() throws NoSuchTableException { insertRows(6); String sql = String.format("SELECT * from %s", tableName); - sql("CREATE VIEW v1 AS %s", sql); - sql("CREATE VIEW prefixV2 AS %s", sql); - sql("CREATE VIEW prefixV3 AS %s", sql); - sql("CREATE GLOBAL TEMPORARY VIEW globalViewForListing AS %s", sql); - sql("CREATE TEMPORARY VIEW tempViewForListing AS %s", sql); + String v1 = viewName("v1"); + String prefixV2 = viewName("prefixV2"); + String prefixV3 = viewName("prefixV3"); + String globalViewForListing = viewName("globalViewForListing"); + String tempViewForListing = viewName("tempViewForListing"); + sql("CREATE VIEW %s AS %s", v1, sql); + sql("CREATE VIEW %s AS %s", prefixV2, sql); + sql("CREATE VIEW %s AS %s", prefixV3, sql); + sql("CREATE GLOBAL TEMPORARY VIEW %s AS %s", globalViewForListing, sql); + sql("CREATE TEMPORARY VIEW %s AS %s", tempViewForListing, sql); // spark stores temp views case-insensitive by default - Object[] tempView = row("", "tempviewforlisting", true); + Object[] tempView = row("", tempViewForListing.toLowerCase(Locale.ROOT), true); assertThat(sql("SHOW VIEWS")) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS IN %s", catalogName)) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, NAMESPACE)) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false), - row(NAMESPACE.toString(), "v1", false), + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), tempView); assertThat(sql("SHOW VIEWS LIKE 'pref*'")) .contains( - row(NAMESPACE.toString(), "prefixV2", false), - row(NAMESPACE.toString(), "prefixV3", false)); + row(NAMESPACE.toString(), prefixV2, false), row(NAMESPACE.toString(), prefixV3, false)); assertThat(sql("SHOW VIEWS LIKE 'non-existing'")).isEmpty(); @@ -1462,7 +1467,7 @@ public void showViews() throws NoSuchTableException { assertThat(sql("SHOW VIEWS IN global_temp")) .contains( // spark stores temp views case-insensitive by default - row("global_temp", "globalviewforlisting", true), tempView); + row("global_temp", globalViewForListing.toLowerCase(Locale.ROOT), true), tempView); sql("USE spark_catalog"); assertThat(sql("SHOW VIEWS")).contains(tempView); @@ -1503,7 +1508,7 @@ public void showViewsWithCurrentNamespace() { @Test public void showCreateSimpleView() { - String viewName = "showCreateSimpleView"; + String viewName = viewName("showCreateSimpleView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql("CREATE VIEW %s AS %s", viewName, sql); @@ -1524,7 +1529,7 @@ public void showCreateSimpleView() { @Test public void showCreateComplexView() { - String viewName = "showCreateComplexView"; + String viewName = viewName("showCreateComplexView"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); sql( @@ -1551,7 +1556,7 @@ public void showCreateComplexView() { @Test public void alterViewSetProperties() { - String viewName = "viewWithSetProperties"; + String viewName = viewName("viewWithSetProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1573,7 +1578,7 @@ public void alterViewSetProperties() { @Test public void alterViewSetReservedProperties() { - String viewName = "viewWithSetReservedProperties"; + String viewName = viewName("viewWithSetReservedProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1604,7 +1609,7 @@ public void alterViewSetReservedProperties() { @Test public void alterViewUnsetProperties() { - String viewName = "viewWithUnsetProperties"; + String viewName = viewName("viewWithUnsetProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); ViewCatalog viewCatalog = viewCatalog(); @@ -1625,7 +1630,7 @@ public void alterViewUnsetProperties() { @Test public void alterViewUnsetUnknownProperty() { - String viewName = "viewWithUnsetUnknownProp"; + String viewName = viewName("viewWithUnsetUnknownProp"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); assertThatThrownBy(() -> sql("ALTER VIEW %s UNSET TBLPROPERTIES ('unknown-key')", viewName)) @@ -1639,7 +1644,7 @@ public void alterViewUnsetUnknownProperty() { @Test public void alterViewUnsetReservedProperties() { - String viewName = "viewWithUnsetReservedProperties"; + String viewName = viewName("viewWithUnsetReservedProperties"); sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1722,7 +1727,7 @@ public void createOrReplaceViewWithColumnAliases() throws NoSuchTableException { @Test public void alterViewIsNotSupported() throws NoSuchTableException { insertRows(6); - String viewName = "alteredView"; + String viewName = viewName("alteredView"); sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); From 9c0a8068462720992b955442b5324ef9b0afee83 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Wed, 23 Oct 2024 10:23:53 -0700 Subject: [PATCH 032/313] Spark 3.4: Action to remove dangling deletes (#11377) Backport #9724 to Spark 3.4 --- .../iceberg/spark/SparkContentFile.java | 7 +- .../RemoveDanglingDeletesSparkAction.java | 171 +++++++ .../actions/RewriteDataFilesSparkAction.java | 33 +- .../iceberg/spark/actions/SparkActions.java | 6 + .../TestRemoveDanglingDeleteAction.java | 426 ++++++++++++++++++ .../actions/TestRewriteDataFilesAction.java | 193 +++++++- 6 files changed, 824 insertions(+), 12 deletions(-) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java create mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java index f756c4cde015..99586f2503c2 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java @@ -52,6 +52,7 @@ public abstract class SparkContentFile implements ContentFile { private final int keyMetadataPosition; private final int splitOffsetsPosition; private final int sortOrderIdPosition; + private final int fileSpecIdPosition; private final int equalityIdsPosition; private final Type lowerBoundsType; private final Type upperBoundsType; @@ -100,6 +101,7 @@ public abstract class SparkContentFile implements ContentFile { this.keyMetadataPosition = positions.get(DataFile.KEY_METADATA.name()); this.splitOffsetsPosition = positions.get(DataFile.SPLIT_OFFSETS.name()); this.sortOrderIdPosition = positions.get(DataFile.SORT_ORDER_ID.name()); + this.fileSpecIdPosition = positions.get(DataFile.SPEC_ID.name()); this.equalityIdsPosition = positions.get(DataFile.EQUALITY_IDS.name()); } @@ -120,7 +122,10 @@ public Long pos() { @Override public int specId() { - return -1; + if (wrapped.isNullAt(fileSpecIdPosition)) { + return -1; + } + return wrapped.getAs(fileSpecIdPosition); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java new file mode 100644 index 000000000000..b9dc46f5e1bc --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.min; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ImmutableRemoveDanglingDeleteFiles; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.SparkDeleteFile; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An action that removes dangling delete files from the current snapshot. A delete file is dangling + * if its deletes no longer applies to any live data files. + * + *

The following dangling delete files are removed: + * + *

    + *
  • Position delete files with a data sequence number less than that of any data file in the + * same partition + *
  • Equality delete files with a data sequence number less than or equal to that of any data + * file in the same partition + *
+ */ +class RemoveDanglingDeletesSparkAction + extends BaseSnapshotUpdateSparkAction + implements RemoveDanglingDeleteFiles { + private static final Logger LOG = LoggerFactory.getLogger(RemoveDanglingDeletesSparkAction.class); + private final Table table; + + protected RemoveDanglingDeletesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + } + + @Override + protected RemoveDanglingDeletesSparkAction self() { + return this; + } + + public Result execute() { + if (table.specs().size() == 1 && table.spec().isUnpartitioned()) { + // ManifestFilterManager already performs this table-wide delete on each commit + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(Collections.emptyList()) + .build(); + } + String desc = String.format("Removing dangling delete files in %s", table.name()); + JobGroupInfo info = newJobGroupInfo("REMOVE-DELETES", desc); + return withJobGroupInfo(info, this::doExecute); + } + + Result doExecute() { + RewriteFiles rewriteFiles = table.newRewrite(); + List danglingDeletes = findDanglingDeletes(); + for (DeleteFile deleteFile : danglingDeletes) { + LOG.debug("Removing dangling delete file {}", deleteFile.path()); + rewriteFiles.deleteFile(deleteFile); + } + if (!danglingDeletes.isEmpty()) { + commit(rewriteFiles); + } + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(danglingDeletes) + .build(); + } + + /** + * Dangling delete files can be identified with following steps + * + *
    + *
  1. Group data files by partition keys and find the minimum data sequence number in each + * group. + *
  2. Left outer join delete files with partition-grouped data files on partition keys. + *
  3. Find dangling deletes by comparing each delete file's sequence number to its partition's + * minimum data sequence number. + *
  4. Collect results row to driver and use {@link SparkDeleteFile SparkDeleteFile} to wrap + * rows to valid delete files + *
+ */ + private List findDanglingDeletes() { + Dataset minSequenceNumberByPartition = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live data files + .filter("data_file.content == 0 AND status < 2") + .selectExpr( + "data_file.partition as partition", + "data_file.spec_id as spec_id", + "sequence_number") + .groupBy("partition", "spec_id") + .agg(min("sequence_number")) + .toDF("grouped_partition", "grouped_spec_id", "min_data_sequence_number"); + Dataset deleteEntries = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live delete files + .filter("data_file.content != 0 AND status < 2"); + Column joinOnPartition = + deleteEntries + .col("data_file.spec_id") + .equalTo(minSequenceNumberByPartition.col("grouped_spec_id")) + .and( + deleteEntries + .col("data_file.partition") + .equalTo(minSequenceNumberByPartition.col("grouped_partition"))); + Column filterOnDanglingDeletes = + col("min_data_sequence_number") + // delete fies without any data files in partition + .isNull() + // position delete files without any applicable data files in partition + .or( + col("data_file.content") + .equalTo("1") + .and(col("sequence_number").$less(col("min_data_sequence_number")))) + // equality delete files without any applicable data files in the partition + .or( + col("data_file.content") + .equalTo("2") + .and(col("sequence_number").$less$eq(col("min_data_sequence_number")))); + Dataset danglingDeletes = + deleteEntries + .join(minSequenceNumberByPartition, joinOnPartition, "left") + .filter(filterOnDanglingDeletes) + .select("data_file.*"); + return danglingDeletes.collectAsList().stream() + // map on driver because SparkDeleteFile is not serializable + .map(row -> deleteFileWrapper(danglingDeletes.schema(), row)) + .collect(Collectors.toList()); + } + + private DeleteFile deleteFileWrapper(StructType sparkFileType, Row row) { + int specId = row.getInt(row.fieldIndex("spec_id")); + Types.StructType combinedFileType = DataFile.getType(Partitioning.partitionType(table)); + // Set correct spec id + Types.StructType projection = DataFile.getType(table.specs().get(specId).partitionType()); + return new SparkDeleteFile(combinedFileType, projection, sparkFileType).wrap(row); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java index a4c6642a3edf..0b2bbb3dfc39 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java @@ -40,6 +40,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.FileRewriter; import org.apache.iceberg.actions.ImmutableRewriteDataFiles; +import org.apache.iceberg.actions.ImmutableRewriteDataFiles.Result.Builder; import org.apache.iceberg.actions.RewriteDataFiles; import org.apache.iceberg.actions.RewriteDataFilesCommitManager; import org.apache.iceberg.actions.RewriteFileGroup; @@ -53,6 +54,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Queues; @@ -82,7 +84,8 @@ public class RewriteDataFilesSparkAction TARGET_FILE_SIZE_BYTES, USE_STARTING_SEQUENCE_NUMBER, REWRITE_JOB_ORDER, - OUTPUT_SPEC_ID); + OUTPUT_SPEC_ID, + REMOVE_DANGLING_DELETES); private static final RewriteDataFilesSparkAction.Result EMPTY_RESULT = ImmutableRewriteDataFiles.Result.builder().rewriteResults(ImmutableList.of()).build(); @@ -93,6 +96,7 @@ public class RewriteDataFilesSparkAction private int maxConcurrentFileGroupRewrites; private int maxCommits; private boolean partialProgressEnabled; + private boolean removeDanglingDeletes; private boolean useStartingSequenceNumber; private RewriteJobOrder rewriteJobOrder; private FileRewriter rewriter = null; @@ -173,11 +177,17 @@ public RewriteDataFiles.Result execute() { Stream groupStream = toGroupStream(ctx, fileGroupsByPartition); - if (partialProgressEnabled) { - return doExecuteWithPartialProgress(ctx, groupStream, commitManager(startingSnapshotId)); - } else { - return doExecute(ctx, groupStream, commitManager(startingSnapshotId)); + Builder resultBuilder = + partialProgressEnabled + ? doExecuteWithPartialProgress(ctx, groupStream, commitManager(startingSnapshotId)) + : doExecute(ctx, groupStream, commitManager(startingSnapshotId)); + if (removeDanglingDeletes) { + RemoveDanglingDeletesSparkAction action = + new RemoveDanglingDeletesSparkAction(spark(), table); + int removedCount = Iterables.size(action.execute().removedDeleteFiles()); + resultBuilder.removedDeleteFilesCount(removedCount); } + return resultBuilder.build(); } StructLikeMap>> planFileGroups(long startingSnapshotId) { @@ -261,7 +271,7 @@ RewriteDataFilesCommitManager commitManager(long startingSnapshotId) { return new RewriteDataFilesCommitManager(table, startingSnapshotId, useStartingSequenceNumber); } - private Result doExecute( + private Builder doExecute( RewriteExecutionContext ctx, Stream groupStream, RewriteDataFilesCommitManager commitManager) { @@ -323,10 +333,10 @@ private Result doExecute( List rewriteResults = rewrittenGroups.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); - return ImmutableRewriteDataFiles.Result.builder().rewriteResults(rewriteResults).build(); + return ImmutableRewriteDataFiles.Result.builder().rewriteResults(rewriteResults); } - private Result doExecuteWithPartialProgress( + private Builder doExecuteWithPartialProgress( RewriteExecutionContext ctx, Stream groupStream, RewriteDataFilesCommitManager commitManager) { @@ -372,8 +382,7 @@ private Result doExecuteWithPartialProgress( commitResults.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); return ImmutableRewriteDataFiles.Result.builder() .rewriteResults(rewriteResults) - .rewriteFailures(rewriteFailures) - .build(); + .rewriteFailures(rewriteFailures); } Stream toGroupStream( @@ -435,6 +444,10 @@ void validateAndInitOptions() { PropertyUtil.propertyAsBoolean( options(), USE_STARTING_SEQUENCE_NUMBER, USE_STARTING_SEQUENCE_NUMBER_DEFAULT); + removeDanglingDeletes = + PropertyUtil.propertyAsBoolean( + options(), REMOVE_DANGLING_DELETES, REMOVE_DANGLING_DELETES_DEFAULT); + rewriteJobOrder = RewriteJobOrder.fromName( PropertyUtil.propertyAsString(options(), REWRITE_JOB_ORDER, REWRITE_JOB_ORDER_DEFAULT)); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index f845386d30c4..ba9fa2e7b4db 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -21,6 +21,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -102,4 +103,9 @@ public RewritePositionDeleteFilesSparkAction rewritePositionDeletes(Table table) public ComputeTableStats computeTableStats(Table table) { return new ComputeTableStatsSparkAction(spark, table); } + + @Override + public RemoveDanglingDeleteFiles removeDanglingDeleteFiles(Table table) { + return new RemoveDanglingDeletesSparkAction(spark, table); + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java new file mode 100644 index 000000000000..8ac9be00ae90 --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -0,0 +1,426 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Encoders; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import scala.Tuple2; + +public class TestRemoveDanglingDeleteAction extends SparkTestBase { + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.StringType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_A2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_UNPARTITIONED = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-unpartitioned.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_POS_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_EQ_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + private String tableLocation = null; + private Table table; + + @Before + public void before() throws Exception { + File tableDir = temp.newFolder(); + this.tableLocation = tableDir.toURI().toString(); + } + + @After + public void after() { + TABLES.dropTable(tableLocation); + } + + private void setupPartitionedTable() { + this.table = + TABLES.create( + SCHEMA, SPEC, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), tableLocation); + } + + private void setupUnpartitionedTable() { + this.table = + TABLES.create( + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + } + + @Test + public void testPartitionedDeletesWithLesserSeqNo() { + setupPartitionedTable(); + // Add Data Files + table.newAppend().appendFile(FILE_B).appendFile(FILE_C).appendFile(FILE_D).commit(); + // Add Delete Files + table + .newRowDelta() + .addDeletes(FILE_A_POS_DELETES) + .addDeletes(FILE_A2_POS_DELETES) + .addDeletes(FILE_B_POS_DELETES) + .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + // Add More Data Files + table + .newAppend() + .appendFile(FILE_A2) + .appendFile(FILE_B2) + .appendFile(FILE_C2) + .appendFile(FILE_D2) + .commit(); + List> actual = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(3L, FILE_A2.path().toString()), + Tuple2.apply(3L, FILE_B2.path().toString()), + Tuple2.apply(3L, FILE_C2.path().toString()), + Tuple2.apply(3L, FILE_D2.path().toString())); + assertThat(actual).isEqualTo(expected); + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + // All Delete files of the FILE A partition should be removed + // because there are no data files in partition with a lesser sequence number + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::path) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected 4 delete files removed") + .hasSize(4) + .containsExactlyInAnyOrder( + FILE_A_POS_DELETES.path(), + FILE_A2_POS_DELETES.path(), + FILE_A_EQ_DELETES.path(), + FILE_A2_EQ_DELETES.path()); + List> actualAfter = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .filter("status < 2") // live files + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(3L, FILE_A2.path().toString()), + Tuple2.apply(3L, FILE_B2.path().toString()), + Tuple2.apply(3L, FILE_C2.path().toString()), + Tuple2.apply(3L, FILE_D2.path().toString())); + assertThat(actualAfter).isEqualTo(expectedAfter); + } + + @Test + public void testPartitionedDeletesWithEqSeqNo() { + setupPartitionedTable(); + // Add Data Files + table.newAppend().appendFile(FILE_A).appendFile(FILE_C).appendFile(FILE_D).commit(); + // Add Data Files with EQ and POS deletes + table + .newRowDelta() + .addRows(FILE_A2) + .addRows(FILE_B2) + .addRows(FILE_C2) + .addRows(FILE_D2) + .addDeletes(FILE_A_POS_DELETES) + .addDeletes(FILE_A2_POS_DELETES) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(FILE_B_POS_DELETES) + .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + List> actual = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2.path().toString()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_C2.path().toString()), + Tuple2.apply(2L, FILE_D2.path().toString())); + assertThat(actual).isEqualTo(expected); + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + // Eq Delete files of the FILE B partition should be removed + // because there are no data files in partition with a lesser sequence number + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::path) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected two delete files removed") + .hasSize(2) + .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.path(), FILE_B2_EQ_DELETES.path()); + List> actualAfter = + spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .filter("status < 2") // live files + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.path().toString()), + Tuple2.apply(1L, FILE_C.path().toString()), + Tuple2.apply(1L, FILE_D.path().toString()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2.path().toString()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_B2.path().toString()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, FILE_C2.path().toString()), + Tuple2.apply(2L, FILE_D2.path().toString())); + assertThat(actualAfter).isEqualTo(expectedAfter); + } + + @Test + public void testUnpartitionedTable() { + setupUnpartitionedTable(); + table + .newRowDelta() + .addDeletes(FILE_UNPARTITIONED_POS_DELETE) + .addDeletes(FILE_UNPARTITIONED_EQ_DELETE) + .commit(); + table.newAppend().appendFile(FILE_UNPARTITIONED).commit(); + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + assertThat(result.removedDeleteFiles()).as("No-op for unpartitioned tables").isEmpty(); + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index ba173d02498e..656b7358d1a5 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -24,6 +24,7 @@ import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.min; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; @@ -55,6 +56,7 @@ import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RewriteJobOrder; import org.apache.iceberg.RowDelta; @@ -71,7 +73,9 @@ import org.apache.iceberg.actions.SizeBasedDataRewriter; import org.apache.iceberg.actions.SizeBasedFileRewriter; import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteWriter; import org.apache.iceberg.encryption.EncryptedFiles; @@ -84,6 +88,7 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -104,9 +109,11 @@ import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.util.ArrayUtil; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.internal.SQLConf; import org.junit.Assert; @@ -129,6 +136,8 @@ public class TestRewriteDataFilesAction extends SparkTestBase { optional(2, "c2", Types.StringType.get()), optional(3, "c3", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + @Rule public TemporaryFolder temp = new TemporaryFolder(); private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); @@ -330,6 +339,108 @@ public void testBinPackWithDeletes() { Assert.assertEquals("7 rows are removed", total - 7, actualRecords.size()); } + @Test + public void testRemoveDangledEqualityDeletesPartitionEvolution() { + Table table = + TABLES.create( + SCHEMA, + SPEC, + Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + // data seq = 1, write 4 files in 2 partitions + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(0, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(0, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + table.refresh(); + shouldHaveFiles(table, 4); + // data seq = 2 & 3, write 2 equality deletes in both partitions + writeEqDeleteRecord(table, "c1", 1, "c3", "AAAA"); + writeEqDeleteRecord(table, "c1", 2, "c3", "CCCC"); + table.refresh(); + Set existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes) + .as("Only one equality delete c1=1 is used in query planning") + .hasSize(1); + // partition evolution + table.refresh(); + table.updateSpec().addField(Expressions.ref("c3")).commit(); + // data seq = 4, write 2 new data files in both partitions for evolved spec + List records3 = + Lists.newArrayList( + new ThreeColumnRecord(1, "A", "CCCC"), new ThreeColumnRecord(2, "D", "DDDD")); + writeRecords(records3); + List originalData = currentData(); + RewriteDataFiles.Result result = + basicRewrite(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .filter(Expressions.equal("c1", 1)) + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes).as("Shall pruned dangling deletes after rewrite").hasSize(0); + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should compact 3 data files into 2 and remove both dangled equality delete file") + .containsExactly(2, 3, 2); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 5); + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + shouldHaveSnapshots(table, 7); + shouldHaveFiles(table, 5); + } + + @Test + public void testRemoveDangledPositionDeletesPartitionEvolution() { + Table table = + TABLES.create( + SCHEMA, + SPEC, + Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + tableLocation); + // data seq = 1, write 4 files in 2 partitions + writeRecords(2, 2, 2); + List dataFilesBefore = TestHelpers.dataFiles(table, null); + shouldHaveFiles(table, 4); + // data seq = 2, write 1 position deletes in c1=1 + table + .newRowDelta() + .addDeletes(writePosDeletesToFile(table, dataFilesBefore.get(3), 1).get(0)) + .commit(); + // partition evolution + table.updateSpec().addField(Expressions.ref("c3")).commit(); + // data seq = 3, write 1 new data files in c1=1 for evolved spec + writeRecords(1, 1, 1); + shouldHaveFiles(table, 5); + List expectedRecords = currentData(); + Result result = + actions() + .rewriteDataFiles(table) + .filter(Expressions.equal("c1", 1)) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should rewrite 2 data files into 1 and remove 1 dangled position delete file") + .containsExactly(1, 2, 1); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 3); + shouldHaveSnapshots(table, 5); + assertThat(table.currentSnapshot().summary().get("total-position-deletes")).isEqualTo("0"); + assertEquals("Rows must match", expectedRecords, currentData()); + } + @Test public void testBinPackWithDeleteAllData() { Map options = Maps.newHashMap(); @@ -1616,6 +1727,21 @@ protected void shouldHaveFiles(Table table, int numExpected) { Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles); } + protected long shouldHaveMinSequenceNumberInPartition( + Table table, String partitionFilter, long expected) { + long actual = + SparkTableUtil.loadMetadataTable(spark, table, MetadataTableType.ENTRIES) + .filter("status != 2") + .filter(partitionFilter) + .select("sequence_number") + .agg(min("sequence_number")) + .as(Encoders.LONG()) + .collectAsList() + .get(0); + assertThat(actual).as("Did not have the expected min sequence number").isEqualTo(expected); + return actual; + } + protected void shouldHaveSnapshots(Table table, int expectedSnapshots) { table.refresh(); int actualSnapshots = Iterables.size(table.snapshots()); @@ -1812,6 +1938,11 @@ protected int averageFileSize(Table table) { .getAsDouble(); } + private void writeRecords(List records) { + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + writeDF(df); + } + private void writeRecords(int files, int numRecords) { writeRecords(files, numRecords, 0); } @@ -1865,7 +1996,10 @@ private List writePosDeletes( table .io() .newOutputFile( - table.locationProvider().newDataLocation(UUID.randomUUID().toString())); + table + .locationProvider() + .newDataLocation( + FileFormat.PARQUET.addExtension(UUID.randomUUID().toString()))); EncryptedOutputFile encryptedOutputFile = EncryptedFiles.encryptedOutput(outputFile, EncryptionKeyMetadata.EMPTY); @@ -1891,6 +2025,63 @@ private List writePosDeletes( return results; } + private void writeEqDeleteRecord( + Table table, String partCol, Object partVal, String delCol, Object delVal) { + List equalityFieldIds = Lists.newArrayList(table.schema().findField(delCol).fieldId()); + Schema eqDeleteRowSchema = table.schema().select(delCol); + Record partitionRecord = + GenericRecord.create(table.schema().select(partCol)) + .copy(ImmutableMap.of(partCol, partVal)); + Record record = GenericRecord.create(eqDeleteRowSchema).copy(ImmutableMap.of(delCol, delVal)); + writeEqDeleteRecord(table, equalityFieldIds, partitionRecord, eqDeleteRowSchema, record); + } + + private void writeEqDeleteRecord( + Table table, + List equalityFieldIds, + Record partitionRecord, + Schema eqDeleteRowSchema, + Record deleteRecord) { + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PARQUET).build(); + GenericAppenderFactory appenderFactory = + new GenericAppenderFactory( + table.schema(), + table.spec(), + ArrayUtil.toIntArray(equalityFieldIds), + eqDeleteRowSchema, + null); + EncryptedOutputFile file = + createEncryptedOutputFile(createPartitionKey(table, partitionRecord), fileFactory); + EqualityDeleteWriter eqDeleteWriter = + appenderFactory.newEqDeleteWriter( + file, FileFormat.PARQUET, createPartitionKey(table, partitionRecord)); + try (EqualityDeleteWriter clsEqDeleteWriter = eqDeleteWriter) { + clsEqDeleteWriter.write(deleteRecord); + } catch (Exception e) { + throw new RuntimeException(e); + } + table.newRowDelta().addDeletes(eqDeleteWriter.toDeleteFile()).commit(); + } + + private PartitionKey createPartitionKey(Table table, Record record) { + if (table.spec().isUnpartitioned()) { + return null; + } + PartitionKey partitionKey = new PartitionKey(table.spec(), table.schema()); + partitionKey.partition(record); + return partitionKey; + } + + private EncryptedOutputFile createEncryptedOutputFile( + PartitionKey partition, OutputFileFactory fileFactory) { + if (partition == null) { + return fileFactory.newOutputFile(); + } else { + return fileFactory.newOutputFile(partition); + } + } + private SparkActions actions() { return SparkActions.get(); } From 043757c0a1cb79392a3dc81054b75d8cfb3bd95e Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 23 Oct 2024 12:36:34 -0700 Subject: [PATCH 033/313] Spark 3.5: Reset Spark Conf for each test in TestCompressionSettings (#11333) --- .../spark/source/TestCompressionSettings.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java index f569446f772b..217b05b46a7c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java @@ -78,6 +78,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; @@ -146,6 +147,13 @@ public static void startSpark() { TestCompressionSettings.spark = SparkSession.builder().master("local[2]").getOrCreate(); } + @BeforeEach + public void resetSpecificConfigurations() { + spark.conf().unset(COMPRESSION_CODEC); + spark.conf().unset(COMPRESSION_LEVEL); + spark.conf().unset(COMPRESSION_STRATEGY); + } + @AfterEach public void afterEach() { spark.sql(String.format("DROP TABLE IF EXISTS %s", TABLE_NAME)); @@ -191,6 +199,8 @@ public void testWriteDataWithDifferentSetting() throws Exception { spark.conf().set(entry.getKey(), entry.getValue()); } + assertSparkConf(); + df.select("id", "data") .writeTo(TABLE_NAME) .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) @@ -253,4 +263,13 @@ private String getCompressionType(InputFile inputFile) throws Exception { return fileReader.getMetaString(DataFileConstants.CODEC); } } + + private void assertSparkConf() { + String[] propertiesToCheck = {COMPRESSION_CODEC, COMPRESSION_LEVEL, COMPRESSION_STRATEGY}; + for (String prop : propertiesToCheck) { + String expected = properties.getOrDefault(prop, null); + String actual = spark.conf().get(prop, null); + assertThat(actual).isEqualToIgnoringCase(expected); + } + } } From 02a988b09ab5b6e9aaa47c79e5e131313cf983cc Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 24 Oct 2024 05:59:18 +0800 Subject: [PATCH 034/313] Spec: Adds Row Lineage (#11130) Co-authored-by: Ryan Blue --- format/spec.md | 289 ++++++++++++++++++++++++++++++++++--------------- 1 file changed, 203 insertions(+), 86 deletions(-) diff --git a/format/spec.md b/format/spec.md index 2974430a3772..601cbcc3bc4e 100644 --- a/format/spec.md +++ b/format/spec.md @@ -51,7 +51,7 @@ Version 3 of the Iceberg spec extends data types and existing metadata structure * New data types: nanosecond timestamp(tz), unknown * Default value support for columns * Multi-argument transforms for partitioning and sorting - +* Row Lineage tracking ## Goals @@ -322,16 +322,101 @@ Iceberg tables must not use field ids greater than 2147483447 (`Integer.MAX_VALU The set of metadata columns is: -| Field id, name | Type | Description | -|-----------------------------|---------------|-------------| -| **`2147483646 _file`** | `string` | Path of the file in which a row is stored | -| **`2147483645 _pos`** | `long` | Ordinal position of a row in the source data file | -| **`2147483644 _deleted`** | `boolean` | Whether the row has been deleted | -| **`2147483643 _spec_id`** | `int` | Spec ID used to track the file containing a row | -| **`2147483642 _partition`** | `struct` | Partition to which a row belongs | -| **`2147483546 file_path`** | `string` | Path of a file, used in position-based delete files | -| **`2147483545 pos`** | `long` | Ordinal position of a row, used in position-based delete files | -| **`2147483544 row`** | `struct<...>` | Deleted row values, used in position-based delete files | +| Field id, name | Type | Description | +|----------------------------------|---------------|--------------------------------------------------------------------------------------------------------| +| **`2147483646 _file`** | `string` | Path of the file in which a row is stored | +| **`2147483645 _pos`** | `long` | Ordinal position of a row in the source data file, starting at `0` | +| **`2147483644 _deleted`** | `boolean` | Whether the row has been deleted | +| **`2147483643 _spec_id`** | `int` | Spec ID used to track the file containing a row | +| **`2147483642 _partition`** | `struct` | Partition to which a row belongs | +| **`2147483546 file_path`** | `string` | Path of a file, used in position-based delete files | +| **`2147483545 pos`** | `long` | Ordinal position of a row, used in position-based delete files | +| **`2147483544 row`** | `struct<...>` | Deleted row values, used in position-based delete files | +| **`2147483543 _row_id`** | `long` | A unique long assigned when row-lineage is enabled, see [Row Lineage](#row-lineage) | +| **`2147483542 _last_updated_sequence_number`** | `long` | The sequence number which last updated this row when row-lineage is enabled [Row Lineage](#row-lineage) | + +### Row Lineage + +In v3 and later, an Iceberg table can track row lineage fields for all newly created rows. Row lineage is enabled by setting the field `row-lineage` to true in the table's metadata. When enabled, engines must maintain the `next-row-id` table field and the following row-level fields when writing data files: + +* `_row_id` a unique long identifier for every row within the table. The value is assigned via inheritance when a row is first added to the table and the existing value is explicitly written when the row is copied into a new file. +* `_last_updated_sequence_number` the sequence number of the commit that last updated a row. The value is inherited when a row is first added or modified and the existing value is explicitly written when the row is written to a different data file but not modified. + +These fields are assigned and updated by inheritance because the commit sequence number and starting row ID are not assigned until the snapshot is successfully committed. Inheritance is used to allow writing data and manifest files before values are known so that it is not necessary to rewrite data and manifest files when an optimistic commit is retried. + +When row lineage is enabled, new snapshots cannot include [Equality Deletes](#equality-delete-files). Row lineage is incompatible with equality deletes because lineage values must be maintained, but equality deletes are used to avoid reading existing data before writing changes. + + +#### Row lineage assignment + +Row lineage fields are written when row lineage is enabled. When not enabled, row lineage fields (`_row_id` and `_last_updated_sequence_number`) must not be written to data files. The rest of this section applies when row lineage is enabled. + +When a row is added or modified, the `_last_updated_sequence_number` field is set to `null` so that it is inherited when reading. Similarly, the `_row_id` field for an added row is set to `null` and assigned when reading. + +A data file with only new rows for the table may omit the `_last_updated_sequence_number` and `_row_id`. If the columns are missing, readers should treat both columns as if they exist and are set to null for all rows. + +On read, if `_last_updated_sequence_number` is `null` it is assigned the `sequence_number` of the data file's manifest entry. The data sequence number of a data file is documented in [Sequence Number Inheritance](#sequence-number-inheritance). + +When `null`, a row's `_row_id` field is assigned to the `first_row_id` from its containing data file plus the row position in that data file (`_pos`). A data file's `first_row_id` field is assigned using inheritance and is documented in [First Row ID Inheritance](#first-row-id-inheritance). A manifest's `first_row_id` is assigned when writing the manifest list for a snapshot and is documented in [First Row ID Assignment](#first-row-id-assignment). A snapshot's `first-row-id` is set to the table's `next-row-id` and is documented in [Snapshot Row IDs](#snapshot-row-ids). + +Values for `_row_id` and `_last_updated_sequence_number` are either read from the data file or assigned at read time. As a result on read, rows in a table always have non-null values for these fields when lineage is enabled. + +When an existing row is moved to a different data file for any reason, writers are required to write `_row_id` and `_last_updated_sequence_number` according to the following rules: + +1. The row's existing non-null `_row_id` must be copied into the new data file +2. If the write has modified the row, the `_last_updated_sequence_number` field must be set to `null` (so that the modification's sequence number replaces the current value) +3. If the write has not modified the row, the existing non-null `_last_updated_sequence_number` value must be copied to the new data file + + +#### Row lineage example + +This example demonstrates how `_row_id` and `_last_updated_sequence_number` are assigned for a snapshot when row lineage is enabled. This starts with a table with row lineage enabled and a `next-row-id` of 1000. + +Writing a new append snapshot would create snapshot metadata with `first-row-id` assigned to the table's `next-row-id`: + +```json +{ + "operation": "append", + "first-row-id": 1000, + ... +} +``` + +The snapshot's manifest list would contain existing manifests, plus new manifests with an assigned `first_row_id` based on the `added_rows_count` of previously listed added manifests: + +| `manifest_path` | `added_rows_count` | `existing_rows_count` | `first_row_id` | +|-----------------|--------------------|-----------------------|--------------------| +| ... | ... | ... | ... | +| existing | 75 | 0 | 925 | +| added1 | 100 | 25 | 1000 | +| added2 | 0 | 100 | 1100 | +| added3 | 125 | 25 | 1100 | + +The first added file, `added1`, is assigned the same `first_row_id` as the snapshot and the following manifests are assigned `first_row_id` based on the number of rows added by the previously listed manifests. The second file, `added2`, does not change the `first_row_id` of the next manifest because it contains no added data files. + +Within `added1`, the first added manifest, each data file's `first_row_id` follows a similar pattern: + +| `status` | `file_path` | `record_count` | `first_row_id` | +|----------|-------------|----------------|----------------| +| EXISTING | data1 | 25 | 800 | +| ADDED | data2 | 50 | null (1000) | +| ADDED | data3 | 50 | null (1050) | + +The `first_row_id` of the EXISTING file `data1` was already assigned, so the file metadata was copied into manifest `added1`. + +Files `data2` and `data3` are written with `null` for `first_row_id` and are assigned `first_row_id` at read time based on the manifest's `first_row_id` and the `record_count` of previously listed ADDED files in this manifest: (1,000 + 0) and (1,000 + 50). + +When the new snapshot is committed, the table's `next-row-id` must also be updated (even if the new snapshot is not in the main branch). Because 225 rows were added (`added1`: 100 + `added2`: 0 + `added3`: 125), the new value is 1,000 + 225 = 1,225: + + +### Enabling Row Lineage for Non-empty Tables + +Any snapshot without the field `first-row-id` does not have any lineage information and values for `_row_id` and `_last_updated_sequence_number` cannot be assigned accurately. + +All files that were added before `row-lineage` was enabled should propagate null for all of the `row-lineage` related +fields. The values for `_row_id` and `_last_updated_sequence_number` should always return null and when these rows are copied, +null should be explicitly written. After this point, rows are treated as if they were just created +and assigned `row_id` and `_last_updated_sequence_number` as if they were new rows. ## Partitioning @@ -478,29 +563,29 @@ The schema of a manifest file is a struct called `manifest_entry` with the follo `data_file` is a struct with the following fields: -| v1 | v2 | Field id, name | Type | Description | -| ---------- | ---------- |-----------------------------------|------------------------------|-------------| -| | _required_ | **`134 content`** | `int` with meaning: `0: DATA`, `1: POSITION DELETES`, `2: EQUALITY DELETES` | Type of content stored by the data file: data, equality deletes, or position deletes (all v1 files are data files) | -| _required_ | _required_ | **`100 file_path`** | `string` | Full URI for the file with FS scheme | -| _required_ | _required_ | **`101 file_format`** | `string` | String file format name, avro, orc or parquet | -| _required_ | _required_ | **`102 partition`** | `struct<...>` | Partition data tuple, schema based on the partition spec output using partition field ids for the struct field ids | -| _required_ | _required_ | **`103 record_count`** | `long` | Number of records in this file | -| _required_ | _required_ | **`104 file_size_in_bytes`** | `long` | Total file size in bytes | -| _required_ | | ~~**`105 block_size_in_bytes`**~~ | `long` | **Deprecated. Always write a default in v1. Do not write in v2.** | -| _optional_ | | ~~**`106 file_ordinal`**~~ | `int` | **Deprecated. Do not write.** | -| _optional_ | | ~~**`107 sort_columns`**~~ | `list<112: int>` | **Deprecated. Do not write.** | -| _optional_ | _optional_ | **`108 column_sizes`** | `map<117: int, 118: long>` | Map from column id to the total size on disk of all regions that store the column. Does not include bytes necessary to read other columns, like footers. Leave null for row-oriented formats (Avro) | -| _optional_ | _optional_ | **`109 value_counts`** | `map<119: int, 120: long>` | Map from column id to number of values in the column (including null and NaN values) | -| _optional_ | _optional_ | **`110 null_value_counts`** | `map<121: int, 122: long>` | Map from column id to number of null values in the column | -| _optional_ | _optional_ | **`137 nan_value_counts`** | `map<138: int, 139: long>` | Map from column id to number of NaN values in the column | -| _optional_ | _optional_ | **`111 distinct_counts`** | `map<123: int, 124: long>` | Map from column id to number of distinct values in the column; distinct counts must be derived using values in the file by counting or using sketches, but not using methods like merging existing distinct counts | -| _optional_ | _optional_ | **`125 lower_bounds`** | `map<126: int, 127: binary>` | Map from column id to lower bound in the column serialized as binary [1]. Each value must be less than or equal to all non-null, non-NaN values in the column for the file [2] | -| _optional_ | _optional_ | **`128 upper_bounds`** | `map<129: int, 130: binary>` | Map from column id to upper bound in the column serialized as binary [1]. Each value must be greater than or equal to all non-null, non-Nan values in the column for the file [2] | -| _optional_ | _optional_ | **`131 key_metadata`** | `binary` | Implementation-specific key metadata for encryption | -| _optional_ | _optional_ | **`132 split_offsets`** | `list<133: long>` | Split offsets for the data file. For example, all row group offsets in a Parquet file. Must be sorted ascending | -| | _optional_ | **`135 equality_ids`** | `list<136: int>` | Field ids used to determine row equality in equality delete files. Required when `content=2` and should be null otherwise. Fields with ids listed in this column must be present in the delete file | -| _optional_ | _optional_ | **`140 sort_order_id`** | `int` | ID representing sort order for this file [3]. | - +| v1 | v2 | v3 | Field id, name | Type | Description | +| ---------- |------------|------------|-----------------------------------|-----------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| | _required_ | _required_ | **`134 content`** | `int` with meaning: `0: DATA`, `1: POSITION DELETES`, `2: EQUALITY DELETES` | Type of content stored by the data file: data, equality deletes, or position deletes (all v1 files are data files) | +| _required_ | _required_ | _required_ | **`100 file_path`** | `string` | Full URI for the file with FS scheme | +| _required_ | _required_ | _required_ | **`101 file_format`** | `string` | String file format name, avro, orc or parquet | +| _required_ | _required_ | _required_ | **`102 partition`** | `struct<...>` | Partition data tuple, schema based on the partition spec output using partition field ids for the struct field ids | +| _required_ | _required_ | _required_ | **`103 record_count`** | `long` | Number of records in this file | +| _required_ | _required_ | _required_ | **`104 file_size_in_bytes`** | `long` | Total file size in bytes | +| _required_ | | | ~~**`105 block_size_in_bytes`**~~ | `long` | **Deprecated. Always write a default in v1. Do not write in v2 or v3.** | +| _optional_ | | | ~~**`106 file_ordinal`**~~ | `int` | **Deprecated. Do not write.** | +| _optional_ | | | ~~**`107 sort_columns`**~~ | `list<112: int>` | **Deprecated. Do not write.** | +| _optional_ | _optional_ | _optional_ | **`108 column_sizes`** | `map<117: int, 118: long>` | Map from column id to the total size on disk of all regions that store the column. Does not include bytes necessary to read other columns, like footers. Leave null for row-oriented formats (Avro) | +| _optional_ | _optional_ | _optional_ | **`109 value_counts`** | `map<119: int, 120: long>` | Map from column id to number of values in the column (including null and NaN values) | +| _optional_ | _optional_ | _optional_ | **`110 null_value_counts`** | `map<121: int, 122: long>` | Map from column id to number of null values in the column | +| _optional_ | _optional_ | _optional_ | **`137 nan_value_counts`** | `map<138: int, 139: long>` | Map from column id to number of NaN values in the column | +| _optional_ | _optional_ | _optional_ | **`111 distinct_counts`** | `map<123: int, 124: long>` | Map from column id to number of distinct values in the column; distinct counts must be derived using values in the file by counting or using sketches, but not using methods like merging existing distinct counts | +| _optional_ | _optional_ | _optional_ | **`125 lower_bounds`** | `map<126: int, 127: binary>` | Map from column id to lower bound in the column serialized as binary [1]. Each value must be less than or equal to all non-null, non-NaN values in the column for the file [2] | +| _optional_ | _optional_ | _optional_ | **`128 upper_bounds`** | `map<129: int, 130: binary>` | Map from column id to upper bound in the column serialized as binary [1]. Each value must be greater than or equal to all non-null, non-Nan values in the column for the file [2] | +| _optional_ | _optional_ | _optional_ | **`131 key_metadata`** | `binary` | Implementation-specific key metadata for encryption | +| _optional_ | _optional_ | _optional_ | **`132 split_offsets`** | `list<133: long>` | Split offsets for the data file. For example, all row group offsets in a Parquet file. Must be sorted ascending | +| | _optional_ | _optional_ | **`135 equality_ids`** | `list<136: int>` | Field ids used to determine row equality in equality delete files. Required when `content=2` and should be null otherwise. Fields with ids listed in this column must be present in the delete file | +| _optional_ | _optional_ | _optional_ | **`140 sort_order_id`** | `int` | ID representing sort order for this file [3]. | +| | | _optional_ | **`142 first_row_id`** | `long` | The `_row_id` for the first row in the data file. See [First Row ID Inheritance](#first-row-id-inheritance) | Notes: 1. Single-value serialization for lower and upper bounds is detailed in Appendix D. @@ -544,21 +629,31 @@ Inheriting sequence numbers through the metadata tree allows writing a new manif When reading v1 manifests with no sequence number column, sequence numbers for all files must default to 0. +### First Row ID Inheritance + +Row ID inheritance is used when row lineage is enabled. When not enabled, a data file's `first_row_id` must always be set to `null`. The rest of this section applies when row lineage is enabled. + +When adding a new data file, its `first_row_id` field is set to `null` because it is not assigned until the snapshot is successfully committed. + +When reading, the `first_row_id` is assigned by replacing `null` with the manifest's `first_row_id` plus the sum of `record_count` for all added data files that preceded the file in the manifest. + +The `first_row_id` is only inherited for added data files. The inherited value must be written into the data file metadata for existing and deleted entries. The value of `first_row_id` for delete files is always `null`. ## Snapshots A snapshot consists of the following fields: -| v1 | v2 | Field | Description | -| ---------- | ---------- | ------------------------ | ----------- | -| _required_ | _required_ | **`snapshot-id`** | A unique long ID | -| _optional_ | _optional_ | **`parent-snapshot-id`** | The snapshot ID of the snapshot's parent. Omitted for any snapshot with no parent | -| | _required_ | **`sequence-number`** | A monotonically increasing long that tracks the order of changes to a table | -| _required_ | _required_ | **`timestamp-ms`** | A timestamp when the snapshot was created, used for garbage collection and table inspection | -| _optional_ | _required_ | **`manifest-list`** | The location of a manifest list for this snapshot that tracks manifest files with additional metadata | -| _optional_ | | **`manifests`** | A list of manifest file locations. Must be omitted if `manifest-list` is present | -| _optional_ | _required_ | **`summary`** | A string map that summarizes the snapshot changes, including `operation` (see below) | -| _optional_ | _optional_ | **`schema-id`** | ID of the table's current schema when the snapshot was created | +| v1 | v2 | v3 | Field | Description | +| ---------- | ---------- |------------|------------------------------|------------------------------------------------------------------------------------------------------------------------------------| +| _required_ | _required_ | _required_ | **`snapshot-id`** | A unique long ID | +| _optional_ | _optional_ | _optional_ | **`parent-snapshot-id`** | The snapshot ID of the snapshot's parent. Omitted for any snapshot with no parent | +| | _required_ | _required_ | **`sequence-number`** | A monotonically increasing long that tracks the order of changes to a table | +| _required_ | _required_ | _required_ | **`timestamp-ms`** | A timestamp when the snapshot was created, used for garbage collection and table inspection | +| _optional_ | _required_ | _required_ | **`manifest-list`** | The location of a manifest list for this snapshot that tracks manifest files with additional metadata | +| _optional_ | | | **`manifests`** | A list of manifest file locations. Must be omitted if `manifest-list` is present | +| _optional_ | _required_ | _required_ | **`summary`** | A string map that summarizes the snapshot changes, including `operation` (see below) | +| _optional_ | _optional_ | _optional_ | **`schema-id`** | ID of the table's current schema when the snapshot was created | +| | | _optional_ | **`first-row-id`** | The first `_row_id` assigned to the first row in the first data file in the first manifest, see [Row Lineage](#row-lineage) | The snapshot summary's `operation` field is used by some operations, like snapshot expiration, to skip processing certain snapshots. Possible `operation` values are: @@ -578,6 +673,15 @@ Manifests for a snapshot are tracked by a manifest list. Valid snapshots are stored as a list in table metadata. For serialization, see Appendix C. +### Snapshot Row IDs + +When row lineage is not enabled, `first-row-id` must be omitted. The rest of this section applies when row lineage is enabled. + +A snapshot's `first-row-id` is assigned to the table's current `next-row-id` on each commit attempt. If a commit is retried, the `first-row-id` must be reassigned. If a commit contains no new rows, `first-row-id` should be omitted. + +The snapshot's `first-row-id` is the starting `first_row_id` assigned to manifests in the snapshot's manifest list. + + ### Manifest Lists Snapshots are embedded in table metadata, but the list of manifests for a snapshot are stored in a separate manifest list file. @@ -590,23 +694,24 @@ A manifest list is a valid Iceberg data file: files must use valid Iceberg forma Manifest list files store `manifest_file`, a struct with the following fields: -| v1 | v2 | Field id, name | Type | Description | -| ---------- | ---------- |--------------------------------|---------------------------------------------|-------------| -| _required_ | _required_ | **`500 manifest_path`** | `string` | Location of the manifest file | -| _required_ | _required_ | **`501 manifest_length`** | `long` | Length of the manifest file in bytes | -| _required_ | _required_ | **`502 partition_spec_id`** | `int` | ID of a partition spec used to write the manifest; must be listed in table metadata `partition-specs` | -| | _required_ | **`517 content`** | `int` with meaning: `0: data`, `1: deletes` | The type of files tracked by the manifest, either data or delete files; 0 for all v1 manifests | -| | _required_ | **`515 sequence_number`** | `long` | The sequence number when the manifest was added to the table; use 0 when reading v1 manifest lists | -| | _required_ | **`516 min_sequence_number`** | `long` | The minimum data sequence number of all live data or delete files in the manifest; use 0 when reading v1 manifest lists | -| _required_ | _required_ | **`503 added_snapshot_id`** | `long` | ID of the snapshot where the manifest file was added | -| _optional_ | _required_ | **`504 added_files_count`** | `int` | Number of entries in the manifest that have status `ADDED` (1), when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`505 existing_files_count`** | `int` | Number of entries in the manifest that have status `EXISTING` (0), when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`506 deleted_files_count`** | `int` | Number of entries in the manifest that have status `DELETED` (2), when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`512 added_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `ADDED`, when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`513 existing_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `EXISTING`, when `null` this is assumed to be non-zero | -| _optional_ | _required_ | **`514 deleted_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `DELETED`, when `null` this is assumed to be non-zero | -| _optional_ | _optional_ | **`507 partitions`** | `list<508: field_summary>` (see below) | A list of field summaries for each partition field in the spec. Each field in the list corresponds to a field in the manifest file’s partition spec. | -| _optional_ | _optional_ | **`519 key_metadata`** | `binary` | Implementation-specific key metadata for encryption | +| v1 | v2 | v3 | Field id, name | Type | Description | +| ---------- | ---------- |------------|----------------------------------|---------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------| +| _required_ | _required_ | _required_ | **`500 manifest_path`** | `string` | Location of the manifest file | +| _required_ | _required_ | _required_ | **`501 manifest_length`** | `long` | Length of the manifest file in bytes | +| _required_ | _required_ | _required_ | **`502 partition_spec_id`** | `int` | ID of a partition spec used to write the manifest; must be listed in table metadata `partition-specs` | +| | _required_ | _required_ | **`517 content`** | `int` with meaning: `0: data`, `1: deletes` | The type of files tracked by the manifest, either data or delete files; 0 for all v1 manifests | +| | _required_ | _required_ | **`515 sequence_number`** | `long` | The sequence number when the manifest was added to the table; use 0 when reading v1 manifest lists | +| | _required_ | _required_ | **`516 min_sequence_number`** | `long` | The minimum data sequence number of all live data or delete files in the manifest; use 0 when reading v1 manifest lists | +| _required_ | _required_ | _required_ | **`503 added_snapshot_id`** | `long` | ID of the snapshot where the manifest file was added | +| _optional_ | _required_ | _required_ | **`504 added_files_count`** | `int` | Number of entries in the manifest that have status `ADDED` (1), when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`505 existing_files_count`** | `int` | Number of entries in the manifest that have status `EXISTING` (0), when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`506 deleted_files_count`** | `int` | Number of entries in the manifest that have status `DELETED` (2), when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`512 added_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `ADDED`, when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`513 existing_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `EXISTING`, when `null` this is assumed to be non-zero | +| _optional_ | _required_ | _required_ | **`514 deleted_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `DELETED`, when `null` this is assumed to be non-zero | +| _optional_ | _optional_ | _optional_ | **`507 partitions`** | `list<508: field_summary>` (see below) | A list of field summaries for each partition field in the spec. Each field in the list corresponds to a field in the manifest file’s partition spec. | +| _optional_ | _optional_ | _optional_ | **`519 key_metadata`** | `binary` | Implementation-specific key metadata for encryption | +| | | _optional_ | **`520 first_row_id`** | `long` | The starting `_row_id` to assign to rows added by `ADDED` data files [First Row ID Assignment](#first-row-id-assignment) | `field_summary` is a struct with the following fields: @@ -622,6 +727,14 @@ Notes: 1. Lower and upper bounds are serialized to bytes using the single-object serialization in Appendix D. The type of used to encode the value is the type of the partition field data. 2. If -0.0 is a value of the partition field, the `lower_bound` must not be +0.0, and if +0.0 is a value of the partition field, the `upper_bound` must not be -0.0. +#### First Row ID Assignment + +Row ID inheritance is used when row lineage is enabled. When not enabled, a manifest's `first_row_id` must always be set to `null`. Once enabled, row lineage cannot be disabled. The rest of this section applies when row lineage is enabled. + +When adding a new data manifest file, its `first_row_id` field is assigned the value of the snapshot's `first_row_id` plus the sum of `added_rows_count` for all data manifests that preceded the manifest in the manifest list. + +The `first_row_id` is only assigned for new data manifests. Values for existing manifests must be preserved when writing a new manifest list. The value of `first_row_id` for delete manifests is always `null`. + ### Scan Planning Scans are planned by reading the manifest files for the current snapshot. Deleted entries in data and delete manifests (those marked with status "DELETED") are not used in a scan. @@ -708,34 +821,38 @@ The atomic operation used to commit metadata depends on how tables are tracked a Table metadata consists of the following fields: -| v1 | v2 | Field | Description | -| ---------- | ---------- | ----- | ----------- | -| _required_ | _required_ | **`format-version`** | An integer version number for the format. Currently, this can be 1 or 2 based on the spec. Implementations must throw an exception if a table's version is higher than the supported version. | -| _optional_ | _required_ | **`table-uuid`** | A UUID that identifies the table, generated when the table is created. Implementations must throw an exception if a table's UUID does not match the expected UUID after refreshing metadata. | -| _required_ | _required_ | **`location`**| The table's base location. This is used by writers to determine where to store data files, manifest files, and table metadata files. | -| | _required_ | **`last-sequence-number`**| The table's highest assigned sequence number, a monotonically increasing long that tracks the order of snapshots in a table. | -| _required_ | _required_ | **`last-updated-ms`**| Timestamp in milliseconds from the unix epoch when the table was last updated. Each table metadata file should update this field just before writing. | -| _required_ | _required_ | **`last-column-id`**| An integer; the highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. | -| _required_ | | **`schema`**| The table’s current schema. (**Deprecated**: use `schemas` and `current-schema-id` instead) | -| _optional_ | _required_ | **`schemas`**| A list of schemas, stored as objects with `schema-id`. | -| _optional_ | _required_ | **`current-schema-id`**| ID of the table's current schema. | -| _required_ | | **`partition-spec`**| The table’s current partition spec, stored as only fields. Note that this is used by writers to partition data, but is not used when reading because reads use the specs stored in manifest files. (**Deprecated**: use `partition-specs` and `default-spec-id` instead) | -| _optional_ | _required_ | **`partition-specs`**| A list of partition specs, stored as full partition spec objects. | -| _optional_ | _required_ | **`default-spec-id`**| ID of the "current" spec that writers should use by default. | -| _optional_ | _required_ | **`last-partition-id`**| An integer; the highest assigned partition field ID across all partition specs for the table. This is used to ensure partition fields are always assigned an unused ID when evolving specs. | -| _optional_ | _optional_ | **`properties`**| A string to string map of table properties. This is used to control settings that affect reading and writing and is not intended to be used for arbitrary metadata. For example, `commit.retry.num-retries` is used to control the number of commit retries. | -| _optional_ | _optional_ | **`current-snapshot-id`**| `long` ID of the current table snapshot; must be the same as the current ID of the `main` branch in `refs`. | -| _optional_ | _optional_ | **`snapshots`**| A list of valid snapshots. Valid snapshots are snapshots for which all data files exist in the file system. A data file must not be deleted from the file system until the last snapshot in which it was listed is garbage collected. | -| _optional_ | _optional_ | **`snapshot-log`**| A list (optional) of timestamp and snapshot ID pairs that encodes changes to the current snapshot for the table. Each time the current-snapshot-id is changed, a new entry should be added with the last-updated-ms and the new current-snapshot-id. When snapshots are expired from the list of valid snapshots, all entries before a snapshot that has expired should be removed. | -| _optional_ | _optional_ | **`metadata-log`**| A list (optional) of timestamp and metadata file location pairs that encodes changes to the previous metadata files for the table. Each time a new metadata file is created, a new entry of the previous metadata file location should be added to the list. Tables can be configured to remove oldest metadata log entries and keep a fixed-size log of the most recent entries after a commit. | -| _optional_ | _required_ | **`sort-orders`**| A list of sort orders, stored as full sort order objects. | -| _optional_ | _required_ | **`default-sort-order-id`**| Default sort order id of the table. Note that this could be used by writers, but is not used when reading because reads use the specs stored in manifest files. | -| | _optional_ | **`refs`** | A map of snapshot references. The map keys are the unique snapshot reference names in the table, and the map values are snapshot reference objects. There is always a `main` branch reference pointing to the `current-snapshot-id` even if the `refs` map is null. | -| _optional_ | _optional_ | **`statistics`** | A list (optional) of [table statistics](#table-statistics). | -| _optional_ | _optional_ | **`partition-statistics`** | A list (optional) of [partition statistics](#partition-statistics). | +| v1 | v2 | v3 | Field | Description | +| ---------- | ---------- |------------|-----------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| _required_ | _required_ | _required_ | **`format-version`** | An integer version number for the format. Currently, this can be 1 or 2 based on the spec. Implementations must throw an exception if a table's version is higher than the supported version. | +| _optional_ | _required_ | _required_ | **`table-uuid`** | A UUID that identifies the table, generated when the table is created. Implementations must throw an exception if a table's UUID does not match the expected UUID after refreshing metadata. | +| _required_ | _required_ | _required_ | **`location`** | The table's base location. This is used by writers to determine where to store data files, manifest files, and table metadata files. | +| | _required_ | _required_ | **`last-sequence-number`** | The table's highest assigned sequence number, a monotonically increasing long that tracks the order of snapshots in a table. | +| _required_ | _required_ | _required_ | **`last-updated-ms`** | Timestamp in milliseconds from the unix epoch when the table was last updated. Each table metadata file should update this field just before writing. | +| _required_ | _required_ | _required_ | **`last-column-id`** | An integer; the highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. | +| _required_ | | | **`schema`** | The table’s current schema. (**Deprecated**: use `schemas` and `current-schema-id` instead) | +| _optional_ | _required_ | _required_ | **`schemas`** | A list of schemas, stored as objects with `schema-id`. | +| _optional_ | _required_ | _required_ | **`current-schema-id`** | ID of the table's current schema. | +| _required_ | | | **`partition-spec`** | The table’s current partition spec, stored as only fields. Note that this is used by writers to partition data, but is not used when reading because reads use the specs stored in manifest files. (**Deprecated**: use `partition-specs` and `default-spec-id` instead) | +| _optional_ | _required_ | _required_ | **`partition-specs`** | A list of partition specs, stored as full partition spec objects. | +| _optional_ | _required_ | _required_ | **`default-spec-id`** | ID of the "current" spec that writers should use by default. | +| _optional_ | _required_ | _required_ | **`last-partition-id`** | An integer; the highest assigned partition field ID across all partition specs for the table. This is used to ensure partition fields are always assigned an unused ID when evolving specs. | +| _optional_ | _optional_ | _optional_ | **`properties`** | A string to string map of table properties. This is used to control settings that affect reading and writing and is not intended to be used for arbitrary metadata. For example, `commit.retry.num-retries` is used to control the number of commit retries. | +| _optional_ | _optional_ | _optional_ | **`current-snapshot-id`** | `long` ID of the current table snapshot; must be the same as the current ID of the `main` branch in `refs`. | +| _optional_ | _optional_ | _optional_ | **`snapshots`** | A list of valid snapshots. Valid snapshots are snapshots for which all data files exist in the file system. A data file must not be deleted from the file system until the last snapshot in which it was listed is garbage collected. | +| _optional_ | _optional_ | _optional_ | **`snapshot-log`** | A list (optional) of timestamp and snapshot ID pairs that encodes changes to the current snapshot for the table. Each time the current-snapshot-id is changed, a new entry should be added with the last-updated-ms and the new current-snapshot-id. When snapshots are expired from the list of valid snapshots, all entries before a snapshot that has expired should be removed. | +| _optional_ | _optional_ | _optional_ | **`metadata-log`** | A list (optional) of timestamp and metadata file location pairs that encodes changes to the previous metadata files for the table. Each time a new metadata file is created, a new entry of the previous metadata file location should be added to the list. Tables can be configured to remove oldest metadata log entries and keep a fixed-size log of the most recent entries after a commit. | +| _optional_ | _required_ | _required_ | **`sort-orders`** | A list of sort orders, stored as full sort order objects. | +| _optional_ | _required_ | _required_ | **`default-sort-order-id`** | Default sort order id of the table. Note that this could be used by writers, but is not used when reading because reads use the specs stored in manifest files. | +| | _optional_ | _optional_ | **`refs`** | A map of snapshot references. The map keys are the unique snapshot reference names in the table, and the map values are snapshot reference objects. There is always a `main` branch reference pointing to the `current-snapshot-id` even if the `refs` map is null. | +| _optional_ | _optional_ | _optional_ | **`statistics`** | A list (optional) of [table statistics](#table-statistics). | +| _optional_ | _optional_ | _optional_ | **`partition-statistics`** | A list (optional) of [partition statistics](#partition-statistics). | +| | | _optional_ | **`row-lineage`** | A boolean, defaulting to false, setting whether or not to track the creation and updates to rows in the table. See [Row Lineage](#row-lineage). | +| | | _optional_ | **`next-row-id`** | A value higher than all assigned row IDs; the next snapshot's `first-row-id`. See [Row Lineage](#row-lineage). | For serialization details, see Appendix C. +When a new snapshot is added, the table's `next-row-id` should be updated to the previous `next-row-id` plus the sum of `record_count` for all data files added in the snapshot (this is also equal to the sum of `added_rows_count` for all manifests added in the snapshot). This ensures that `next-row-id` is always higher than any assigned row ID in the table. + ### Table Statistics Table statistics files are valid [Puffin files](puffin-spec.md). Statistics are informational. A reader can choose to From 4850b622c778deb4b234880bfd7643070e0a5458 Mon Sep 17 00:00:00 2001 From: stubz151 Date: Thu, 24 Oct 2024 04:45:00 +0100 Subject: [PATCH 035/313] AWS: Support S3 directory bucket listing (#11021) --- .../apache/iceberg/aws/AwsIntegTestUtil.java | 47 +++++++- .../apache/iceberg/aws/glue/GlueTestBase.java | 2 +- .../aws/s3/TestS3FileIOIntegration.java | 73 ++++++++++--- .../iceberg/aws/s3/TestS3MultipartUpload.java | 2 +- .../org/apache/iceberg/aws/s3/S3FileIO.java | 8 +- .../iceberg/aws/s3/S3FileIOProperties.java | 37 +++++++ .../java/org/apache/iceberg/aws/s3/S3URI.java | 33 ++++++ .../iceberg/aws/TestS3FileIOProperties.java | 8 ++ .../apache/iceberg/aws/s3/TestS3FileIO.java | 103 +++++++++++++++++- .../org/apache/iceberg/aws/s3/TestS3URI.java | 43 ++++++++ 10 files changed, 333 insertions(+), 23 deletions(-) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java index e9cf474addfa..6b57cfd68243 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java @@ -32,9 +32,11 @@ import software.amazon.awssdk.services.s3.model.Delete; import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; import software.amazon.awssdk.services.s3.model.ListObjectVersionsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import software.amazon.awssdk.services.s3.model.ObjectVersion; import software.amazon.awssdk.services.s3.paginators.ListObjectVersionsIterable; +import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Iterable; import software.amazon.awssdk.services.s3control.S3ControlClient; import software.amazon.awssdk.services.s3control.model.CreateAccessPointRequest; import software.amazon.awssdk.services.s3control.model.DeleteAccessPointRequest; @@ -42,6 +44,7 @@ public class AwsIntegTestUtil { private static final Logger LOG = LoggerFactory.getLogger(AwsIntegTestUtil.class); + private static final int BATCH_DELETION_SIZE = 1000; private AwsIntegTestUtil() {} @@ -106,17 +109,16 @@ public static String testMultiRegionAccessPointAlias() { return System.getenv("AWS_TEST_MULTI_REGION_ACCESS_POINT_ALIAS"); } - public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) { + public static void cleanS3GeneralPurposeBucket(S3Client s3, String bucketName, String prefix) { ListObjectVersionsIterable response = s3.listObjectVersionsPaginator( ListObjectVersionsRequest.builder().bucket(bucketName).prefix(prefix).build()); List versionsToDelete = Lists.newArrayList(); - int batchDeletionSize = 1000; response.versions().stream() .forEach( version -> { versionsToDelete.add(version); - if (versionsToDelete.size() == batchDeletionSize) { + if (versionsToDelete.size() == BATCH_DELETION_SIZE) { deleteObjectVersions(s3, bucketName, versionsToDelete); versionsToDelete.clear(); } @@ -127,6 +129,45 @@ public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) } } + /** + * Method used to clean up a S3 directory bucket which doesn't care about versions + * + * @param s3 an instance of S3Client to be used to list/delete objects + * @param bucketName name of the bucket + * @param prefix the path prefix we want to remove + */ + public static void cleanS3DirectoryBucket(S3Client s3, String bucketName, String prefix) { + String newPrefix = prefix.endsWith("/") ? prefix : prefix + "/"; + ListObjectsV2Request listRequest = + ListObjectsV2Request.builder().bucket(bucketName).prefix(newPrefix).build(); + + ListObjectsV2Iterable paginatedListResponse = s3.listObjectsV2Paginator(listRequest); + List objectsToDelete = Lists.newArrayList(); + + paginatedListResponse.contents().stream() + .forEach( + s3Object -> { + if (objectsToDelete.size() == BATCH_DELETION_SIZE) { + deleteObjects(s3, bucketName, objectsToDelete); + objectsToDelete.clear(); + } + objectsToDelete.add(ObjectIdentifier.builder().key(s3Object.key()).build()); + }); + + if (!objectsToDelete.isEmpty()) { + deleteObjects(s3, bucketName, objectsToDelete); + } + } + + private static void deleteObjects( + S3Client s3, String bucketName, List objectsToDelete) { + s3.deleteObjects( + DeleteObjectsRequest.builder() + .bucket(bucketName) + .delete(Delete.builder().objects(objectsToDelete).build()) + .build()); + } + private static void deleteObjectVersions( S3Client s3, String bucket, List objectVersions) { s3.deleteObjects( diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java index 29076369c8f5..65e37eba4cd3 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java @@ -110,7 +110,7 @@ public static void beforeClass() { @AfterAll public static void afterClass() { AwsIntegTestUtil.cleanGlueCatalog(GLUE, NAMESPACES); - AwsIntegTestUtil.cleanS3Bucket(S3, TEST_BUCKET_NAME, TEST_PATH_PREFIX); + AwsIntegTestUtil.cleanS3GeneralPurposeBucket(S3, TEST_BUCKET_NAME, TEST_PATH_PREFIX); } public static String getRandomName() { diff --git a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java index 41a07401a1e6..9d5d41438a62 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java @@ -108,25 +108,32 @@ public static void beforeClass() { content = new String(contentBytes, StandardCharsets.UTF_8); kmsKeyArn = kms.createKey().keyMetadata().arn(); - AwsIntegTestUtil.createAccessPoint(s3Control, accessPointName, bucketName); - AwsIntegTestUtil.createAccessPoint( - crossRegionS3Control, crossRegionAccessPointName, crossRegionBucketName); - multiRegionAccessPointAlias = AwsIntegTestUtil.testMultiRegionAccessPointAlias(); - s3.putBucketVersioning( - PutBucketVersioningRequest.builder() - .bucket(bucketName) - .versioningConfiguration( - VersioningConfiguration.builder().status(BucketVersioningStatus.ENABLED).build()) - .build()); + if (!S3URI.isS3DirectoryBucket(bucketName)) { + s3.putBucketVersioning( + PutBucketVersioningRequest.builder() + .bucket(bucketName) + .versioningConfiguration( + VersioningConfiguration.builder().status(BucketVersioningStatus.ENABLED).build()) + .build()); + AwsIntegTestUtil.createAccessPoint(s3Control, accessPointName, bucketName); + AwsIntegTestUtil.createAccessPoint( + crossRegionS3Control, crossRegionAccessPointName, crossRegionBucketName); + multiRegionAccessPointAlias = AwsIntegTestUtil.testMultiRegionAccessPointAlias(); + } } @AfterAll public static void afterClass() { - AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix); - AwsIntegTestUtil.deleteAccessPoint(s3Control, accessPointName); - AwsIntegTestUtil.deleteAccessPoint(crossRegionS3Control, crossRegionAccessPointName); - kms.scheduleKeyDeletion( - ScheduleKeyDeletionRequest.builder().keyId(kmsKeyArn).pendingWindowInDays(7).build()); + if (S3URI.isS3DirectoryBucket(bucketName)) { + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + AwsIntegTestUtil.cleanS3DirectoryBucket(s3FileIO.client(), bucketName, prefix); + } else { + AwsIntegTestUtil.cleanS3GeneralPurposeBucket(s3, bucketName, prefix); + AwsIntegTestUtil.deleteAccessPoint(s3Control, accessPointName); + AwsIntegTestUtil.deleteAccessPoint(crossRegionS3Control, crossRegionAccessPointName); + kms.scheduleKeyDeletion( + ScheduleKeyDeletionRequest.builder().keyId(kmsKeyArn).pendingWindowInDays(7).build()); + } } @BeforeEach @@ -171,6 +178,7 @@ public void testS3FileIOWithDefaultAwsClientFactoryImpl() throws Exception { @Test public void testNewInputStreamWithAccessPoint() throws Exception { + requireAccessPointSupport(); s3.putObject( PutObjectRequest.builder().bucket(bucketName).key(objectKey).build(), RequestBody.fromBytes(contentBytes)); @@ -201,12 +209,14 @@ public void testCrossRegionAccessEnabled() throws Exception { S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); validateRead(s3FileIO, crossBucketObjectUri); } finally { - AwsIntegTestUtil.cleanS3Bucket(s3Client, crossRegionBucketName, crossBucketObjectKey); + AwsIntegTestUtil.cleanS3GeneralPurposeBucket( + s3Client, crossRegionBucketName, crossBucketObjectKey); } } @Test public void testNewInputStreamWithCrossRegionAccessPoint() throws Exception { + requireAccessPointSupport(); clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); S3Client s3Client = clientFactory.s3(); s3Client.putObject( @@ -258,6 +268,7 @@ public void testNewOutputStream() throws Exception { @Test public void testNewOutputStreamWithAccessPoint() throws Exception { + requireAccessPointSupport(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); s3FileIO.initialize( ImmutableMap.of( @@ -273,6 +284,7 @@ public void testNewOutputStreamWithAccessPoint() throws Exception { @Test public void testNewOutputStreamWithCrossRegionAccessPoint() throws Exception { + requireAccessPointSupport(); clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); S3Client s3Client = clientFactory.s3(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); @@ -327,6 +339,7 @@ public void testServerSideS3Encryption() throws Exception { @Test public void testServerSideKmsEncryption() throws Exception { + requireKMSEncryptionSupport(); S3FileIOProperties properties = new S3FileIOProperties(); properties.setSseType(S3FileIOProperties.SSE_TYPE_KMS); properties.setSseKey(kmsKeyArn); @@ -342,6 +355,7 @@ public void testServerSideKmsEncryption() throws Exception { @Test public void testServerSideKmsEncryptionWithDefaultKey() throws Exception { + requireKMSEncryptionSupport(); S3FileIOProperties properties = new S3FileIOProperties(); properties.setSseType(S3FileIOProperties.SSE_TYPE_KMS); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties); @@ -363,6 +377,7 @@ public void testServerSideKmsEncryptionWithDefaultKey() throws Exception { @Test public void testDualLayerServerSideKmsEncryption() throws Exception { + requireKMSEncryptionSupport(); S3FileIOProperties properties = new S3FileIOProperties(); properties.setSseType(S3FileIOProperties.DSSE_TYPE_KMS); properties.setSseKey(kmsKeyArn); @@ -378,6 +393,7 @@ public void testDualLayerServerSideKmsEncryption() throws Exception { @Test public void testServerSideCustomEncryption() throws Exception { + requireKMSEncryptionSupport(); // generate key KeyGenerator keyGenerator = KeyGenerator.getInstance("AES"); keyGenerator.init(256, new SecureRandom()); @@ -413,6 +429,7 @@ public void testServerSideCustomEncryption() throws Exception { @Test public void testACL() throws Exception { + requireACLSupport(); S3FileIOProperties properties = new S3FileIOProperties(); properties.setAcl(ObjectCannedACL.BUCKET_OWNER_FULL_CONTROL); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties); @@ -444,6 +461,7 @@ public void testDeleteFilesMultipleBatches() throws Exception { @Test public void testDeleteFilesMultipleBatchesWithAccessPoints() throws Exception { + requireAccessPointSupport(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, getDeletionTestProperties()); s3FileIO.initialize( ImmutableMap.of( @@ -454,6 +472,7 @@ public void testDeleteFilesMultipleBatchesWithAccessPoints() throws Exception { @Test public void testDeleteFilesMultipleBatchesWithCrossRegionAccessPoints() throws Exception { + requireKMSEncryptionSupport(); clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, getDeletionTestProperties()); s3FileIO.initialize( @@ -515,6 +534,7 @@ public void testPrefixDelete() { @Test public void testFileRecoveryHappyPath() throws Exception { + requireVersioningSupport(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, new S3FileIOProperties()); String filePath = String.format("s3://%s/%s/%s", bucketName, prefix, "someFile.parquet"); write(s3FileIO, filePath); @@ -527,6 +547,7 @@ public void testFileRecoveryHappyPath() throws Exception { @Test public void testFileRecoveryFailsToRecover() throws Exception { + requireVersioningSupport(); S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, new S3FileIOProperties()); s3.putBucketVersioning( PutBucketVersioningRequest.builder() @@ -613,4 +634,24 @@ private void createRandomObjects(String objectPrefix, int count) { builder -> builder.bucket(s3URI.bucket()).key(s3URI.key() + i).build(), RequestBody.empty())); } + + /** S3 Express doesn't support access points */ + private void requireAccessPointSupport() { + Assumptions.assumeThat(S3URI.isS3DirectoryBucket(bucketName)).isFalse(); + } + + /** S3 Express doesn’t support KMS/custom encryption */ + private void requireKMSEncryptionSupport() { + Assumptions.assumeThat(S3URI.isS3DirectoryBucket(bucketName)).isFalse(); + } + + /** S3 Express doesn't support versioning */ + private void requireVersioningSupport() { + Assumptions.assumeThat(S3URI.isS3DirectoryBucket(bucketName)).isFalse(); + } + + /** File ACLs aren’t supported by S3 Express */ + private void requireACLSupport() { + Assumptions.assumeThat(S3URI.isS3DirectoryBucket(bucketName)).isFalse(); + } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java index 29d4c48927fc..901e9933b1fd 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3MultipartUpload.java @@ -59,7 +59,7 @@ public static void beforeClass() { @AfterAll public static void afterClass() { - AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix); + AwsIntegTestUtil.cleanS3GeneralPurposeBucket(s3, bucketName, prefix); } @BeforeEach diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java index 11a5ce02247f..23b246c357c9 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java @@ -297,7 +297,13 @@ private List deleteBatch(String bucket, Collection keysToDelete) @Override public Iterable listPrefix(String prefix) { - S3URI s3uri = new S3URI(prefix, s3FileIOProperties.bucketToAccessPointMapping()); + S3URI uri = new S3URI(prefix, s3FileIOProperties.bucketToAccessPointMapping()); + if (uri.useS3DirectoryBucket() + && s3FileIOProperties.isS3DirectoryBucketListPrefixAsDirectory()) { + uri = uri.toDirectoryPath(); + } + + S3URI s3uri = uri; ListObjectsV2Request request = ListObjectsV2Request.builder().bucket(s3uri.bucket()).prefix(s3uri.key()).build(); diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java index 3a43880f31ed..5da758704ae5 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java @@ -428,6 +428,25 @@ public class S3FileIOProperties implements Serializable { public static final long S3_RETRY_MAX_WAIT_MS_DEFAULT = 20_000; // 20 seconds + /** + * Controls whether to list prefixes as directories for S3 Directory buckets Defaults value is + * true, where it will add the "/" + * + *

Example: s3://bucket/prefix will be shown as s3://bucket/prefix/ + * + *

For more details see delimiter section in: + * https://docs.aws.amazon.com/AmazonS3/latest/API/API_ListObjectsV2.html#API_ListObjectsV2_RequestSyntax + * + *

If set to false, this will throw an error when the "/" is not provided for directory bucket. + * Turn off this feature if you are using S3FileIO.listPrefix for listing bucket prefixes that are + * not directories. This would ensure correctness and fail the operation based on S3 requirement + * when listing against a non-directory prefix in a directory bucket. + */ + public static final String S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY = + "s3.directory-bucket.list-prefix-as-directory"; + + public static final boolean S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY_DEFAULT = true; + private String sseType; private String sseKey; private String sseMd5; @@ -462,6 +481,8 @@ public class S3FileIOProperties implements Serializable { private int s3RetryNumRetries; private long s3RetryMinWaitMs; private long s3RetryMaxWaitMs; + + private boolean s3DirectoryBucketListPrefixAsDirectory; private final Map allProperties; public S3FileIOProperties() { @@ -498,6 +519,8 @@ public S3FileIOProperties() { this.s3RetryNumRetries = S3_RETRY_NUM_RETRIES_DEFAULT; this.s3RetryMinWaitMs = S3_RETRY_MIN_WAIT_MS_DEFAULT; this.s3RetryMaxWaitMs = S3_RETRY_MAX_WAIT_MS_DEFAULT; + this.s3DirectoryBucketListPrefixAsDirectory = + S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY_DEFAULT; this.allProperties = Maps.newHashMap(); ValidationException.check( @@ -605,6 +628,11 @@ public S3FileIOProperties(Map properties) { PropertyUtil.propertyAsLong(properties, S3_RETRY_MIN_WAIT_MS, S3_RETRY_MIN_WAIT_MS_DEFAULT); this.s3RetryMaxWaitMs = PropertyUtil.propertyAsLong(properties, S3_RETRY_MAX_WAIT_MS, S3_RETRY_MAX_WAIT_MS_DEFAULT); + this.s3DirectoryBucketListPrefixAsDirectory = + PropertyUtil.propertyAsBoolean( + properties, + S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY, + S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY_DEFAULT); ValidationException.check( keyIdAccessKeyBothConfigured(), @@ -837,6 +865,15 @@ public long s3RetryTotalWaitMs() { return (long) s3RetryNumRetries() * s3RetryMaxWaitMs(); } + public boolean isS3DirectoryBucketListPrefixAsDirectory() { + return s3DirectoryBucketListPrefixAsDirectory; + } + + public void setS3DirectoryBucketListPrefixAsDirectory( + boolean s3DirectoryBucketListPrefixAsDirectory) { + this.s3DirectoryBucketListPrefixAsDirectory = s3DirectoryBucketListPrefixAsDirectory; + } + private boolean keyIdAccessKeyBothConfigured() { return (accessKeyId == null) == (secretAccessKey == null); } diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3URI.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3URI.java index 79b4e695defc..9cfba5fca35c 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3URI.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3URI.java @@ -37,6 +37,9 @@ class S3URI { private static final String QUERY_DELIM = "\\?"; private static final String FRAGMENT_DELIM = "#"; + /** Suffix of S3Express storage bucket names. */ + private static final String S3_DIRECTORY_BUCKET_SUFFIX = "--x-s3"; + private final String location; private final String scheme; private final String bucket; @@ -115,4 +118,34 @@ public String scheme() { public String toString() { return location; } + + /** + * Converts the current S3URI to a directory path. + * + *

This method ensures that the S3URI represents a directory by adding a "/" delimiter at the + * end of the prefix if it's not already present. + * + * @return a S3URI with the directory path configured + */ + public S3URI toDirectoryPath() { + if (key.endsWith(PATH_DELIM)) { + return this; + } + return new S3URI(String.format("%s://%s/%s/", scheme, bucket, key)); + } + + public boolean useS3DirectoryBucket() { + return isS3DirectoryBucket(this.bucket); + } + + /** + * Check if the bucket name indicates the bucket is a directory bucket. This method does not check + * against the S3 service. + * + * @param bucket bucket to probe. + * @return true if the bucket name indicates the bucket is a directory bucket + */ + public static boolean isS3DirectoryBucket(final String bucket) { + return bucket.endsWith(S3_DIRECTORY_BUCKET_SUFFIX); + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java index e2499e947617..58332d42588e 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java @@ -312,4 +312,12 @@ public void testS3AccessGrantsDisabled() { s3Properties.applyS3AccessGrantsConfigurations(builder); assertThat(builder.plugins().size()).isEqualTo(0); } + + @Test + public void testIsTreatS3DirectoryBucketListPrefixAsDirectoryEnabled() { + Map map = Maps.newHashMap(); + map.put(S3FileIOProperties.S3_DIRECTORY_BUCKET_LIST_PREFIX_AS_DIRECTORY, "false"); + S3FileIOProperties properties = new S3FileIOProperties(map); + assertThat(properties.isS3DirectoryBucketListPrefixAsDirectory()).isEqualTo(false); + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java index 6caa42fb410b..77717d796176 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java @@ -20,6 +20,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; @@ -34,10 +36,17 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.time.Instant; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Random; +import java.util.Spliterator; +import java.util.Spliterators; import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; @@ -58,6 +67,7 @@ import org.apache.iceberg.io.BulkDeletionFailureException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.FileIOParser; +import org.apache.iceberg.io.FileInfo; import org.apache.iceberg.io.IOUtil; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; @@ -76,6 +86,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.RegisterExtension; +import org.mockito.Mockito; import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; @@ -86,7 +97,11 @@ import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; import software.amazon.awssdk.services.s3.model.S3Error; +import software.amazon.awssdk.services.s3.model.S3Object; +import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Iterable; @ExtendWith(S3MockExtension.class) public class TestS3FileIO { @@ -101,6 +116,9 @@ public class TestS3FileIO { private final int batchDeletionSize = 5; private S3FileIO s3FileIO; + private static final String S3_GENERAL_PURPOSE_BUCKET = "bucket"; + private static final String S3_DIRECTORY_BUCKET = "directory-bucket-usw2-az1--x-s3"; + private final Map properties = ImmutableMap.of( "s3.write.tags.tagKey1", @@ -112,7 +130,7 @@ public class TestS3FileIO { public void before() { s3FileIO = new S3FileIO(() -> s3mock); s3FileIO.initialize(properties); - createBucket("bucket"); + createBucket(S3_GENERAL_PURPOSE_BUCKET); for (int i = 1; i <= numBucketsForBatchDeletion; i++) { createBucket(batchDeletionBucketPrefix + i); } @@ -257,6 +275,89 @@ public void testPrefixList() { assertThat(Streams.stream(s3FileIO.listPrefix(prefix)).count()).isEqualTo(totalFiles); } + /** + * Tests that we correctly insert the backslash for s3 express buckets. Currently the Adobe S3 + * Mock doesn't cater for express buckets eg. When you call createBucket with s3 express + * configurations it still just returns a general bucket TODO Update to use S3Mock when it behaves + * as expected. + */ + @Test + public void testPrefixListWithExpressAddSlash() { + assertPrefixIsAddedCorrectly("path/to/list", properties); + + Map newProperties = + ImmutableMap.of( + "s3.write.tags.tagKey1", + "TagValue1", + "s3.delete.batch-size", + Integer.toString(batchDeletionSize), + "s3.directory-bucket.list-prefix-as-directory", + "true"); + assertPrefixIsAddedCorrectly("path/to/list/", newProperties); + } + + public void assertPrefixIsAddedCorrectly(String suffix, Map props) { + String prefix = String.format("s3://%s/%s", S3_DIRECTORY_BUCKET, suffix); + + S3Client localMockedClient = mock(S3Client.class); + + List s3Objects = + Arrays.asList( + S3Object.builder() + .key("path/to/list/file1.txt") + .size(1024L) + .lastModified(Instant.now()) + .build(), + S3Object.builder() + .key("path/to/list/file2.txt") + .size(2048L) + .lastModified(Instant.now().minusSeconds(60)) + .build()); + + ListObjectsV2Response response = ListObjectsV2Response.builder().contents(s3Objects).build(); + + ListObjectsV2Iterable mockedResponse = mock(ListObjectsV2Iterable.class); + + Mockito.when(mockedResponse.stream()).thenReturn(Stream.of(response)); + + Mockito.when( + localMockedClient.listObjectsV2Paginator( + ListObjectsV2Request.builder() + .prefix("path/to/list/") + .bucket(S3_DIRECTORY_BUCKET) + .build())) + .thenReturn(mockedResponse); + + // Initialize S3FileIO with the mocked client + S3FileIO localS3FileIo = new S3FileIO(() -> localMockedClient); + localS3FileIo.initialize(props); + + // Perform the listing + List fileInfoList = + StreamSupport.stream( + Spliterators.spliteratorUnknownSize( + localS3FileIo.listPrefix(prefix).iterator(), Spliterator.ORDERED), + false) + .collect(Collectors.toList()); + + // Assert that the returned FileInfo instances match the expected values + assertEquals(2, fileInfoList.size()); + assertTrue( + fileInfoList.stream() + .anyMatch( + fi -> + fi.location().endsWith("file1.txt") + && fi.size() == 1024 + && fi.createdAtMillis() > Instant.now().minusSeconds(120).toEpochMilli())); + assertTrue( + fileInfoList.stream() + .anyMatch( + fi -> + fi.location().endsWith("file2.txt") + && fi.size() == 2048 + && fi.createdAtMillis() < Instant.now().minusSeconds(30).toEpochMilli())); + } + /** * Ignoring because the test is flaky, failing with 500s from S3Mock. Coverage of prefix delete * exists through integration tests. diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3URI.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3URI.java index 383ff67d161d..d3f8ac35d487 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3URI.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3URI.java @@ -28,6 +28,7 @@ import org.junit.jupiter.api.Test; public class TestS3URI { + private static final String S3_DIRECTORY_BUCKET = "directory-bucket-usw2-az1--x-s3"; @Test public void testLocationParsing() { @@ -96,4 +97,46 @@ public void testS3URIWithBucketToAccessPointMapping() { assertThat(uri1.key()).isEqualTo("path/to/file"); assertThat(uri1.toString()).isEqualTo(p1); } + + @Test + public void testS3URIUseS3DirectoryBucket() { + assertThat( + new S3URI(String.format("s3://%s/path/to/file", S3_DIRECTORY_BUCKET)) + .useS3DirectoryBucket()) + .isTrue(); + assertThat(new S3URI("s3://bucket/path/to/file").useS3DirectoryBucket()).isFalse(); + assertThat( + new S3URI("s3://bucket/path/to/file", ImmutableMap.of("bucket", S3_DIRECTORY_BUCKET)) + .useS3DirectoryBucket()) + .isTrue(); + assertThat( + new S3URI("s3://bucket/path/to/file", ImmutableMap.of("bucket", "bucket2")) + .useS3DirectoryBucket()) + .isFalse(); + } + + @Test + public void testS3URIToDirectoryPath() { + assertThat(new S3URI("s3://bucket/path/to/file").toDirectoryPath().location()) + .isEqualTo("s3://bucket/path/to/file/"); + assertThat(new S3URI("s3://bucket/path/to/file/").toDirectoryPath().location()) + .isEqualTo("s3://bucket/path/to/file/"); + assertThat(new S3URI("s3a://bucket/path/to/file").toDirectoryPath().location()) + .isEqualTo("s3a://bucket/path/to/file/"); + assertThat( + new S3URI(String.format("s3://%s/path/to/file", S3_DIRECTORY_BUCKET)) + .toDirectoryPath() + .location()) + .isEqualTo(String.format("s3://%s/path/to/file/", S3_DIRECTORY_BUCKET)); + assertThat( + new S3URI("s3://bucket/path/to/file", ImmutableMap.of("bucket", S3_DIRECTORY_BUCKET)) + .toDirectoryPath() + .location()) + .isEqualTo(String.format("s3://%s/path/to/file/", S3_DIRECTORY_BUCKET)); + assertThat( + new S3URI("s3://bucket/path/to/file", ImmutableMap.of("bucket", "bucket2")) + .toDirectoryPath() + .location()) + .isEqualTo("s3://bucket2/path/to/file/"); + } } From 1cb88a64f8a065c87eb875cf08cfc70941a2fd05 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 24 Oct 2024 12:33:27 +0200 Subject: [PATCH 036/313] Core: Add LoadCredentialsResponse class/parser (#11339) --- .../apache/iceberg/rest/RESTSerializers.java | 29 ++++- .../responses/LoadCredentialsResponse.java | 34 ++++++ .../LoadCredentialsResponseParser.java | 77 ++++++++++++ .../responses/LoadTableResponseParser.java | 8 +- .../responses/LoadViewResponseParser.java | 8 +- .../TestLoadCredentialsResponseParser.java | 112 ++++++++++++++++++ 6 files changed, 253 insertions(+), 15 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponse.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponseParser.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/responses/TestLoadCredentialsResponseParser.java diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java index 7f39d0bc1f5e..667142698633 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java @@ -59,7 +59,10 @@ import org.apache.iceberg.rest.responses.ConfigResponseParser; import org.apache.iceberg.rest.responses.ErrorResponse; import org.apache.iceberg.rest.responses.ErrorResponseParser; +import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; import org.apache.iceberg.rest.responses.ImmutableLoadViewResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponseParser; import org.apache.iceberg.rest.responses.LoadTableResponse; import org.apache.iceberg.rest.responses.LoadTableResponseParser; import org.apache.iceberg.rest.responses.LoadViewResponse; @@ -119,7 +122,13 @@ public static void registerAll(ObjectMapper mapper) { .addSerializer(ConfigResponse.class, new ConfigResponseSerializer<>()) .addDeserializer(ConfigResponse.class, new ConfigResponseDeserializer<>()) .addSerializer(LoadTableResponse.class, new LoadTableResponseSerializer<>()) - .addDeserializer(LoadTableResponse.class, new LoadTableResponseDeserializer<>()); + .addDeserializer(LoadTableResponse.class, new LoadTableResponseDeserializer<>()) + .addSerializer(LoadCredentialsResponse.class, new LoadCredentialsResponseSerializer<>()) + .addSerializer( + ImmutableLoadCredentialsResponse.class, new LoadCredentialsResponseSerializer<>()) + .addDeserializer(LoadCredentialsResponse.class, new LoadCredentialsResponseDeserializer<>()) + .addDeserializer( + ImmutableLoadCredentialsResponse.class, new LoadCredentialsResponseDeserializer<>()); mapper.registerModule(module); } @@ -443,4 +452,22 @@ public T deserialize(JsonParser p, DeserializationContext context) throws IOExce return (T) LoadTableResponseParser.fromJson(jsonNode); } } + + static class LoadCredentialsResponseSerializer + extends JsonSerializer { + @Override + public void serialize(T request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + LoadCredentialsResponseParser.toJson(request, gen); + } + } + + static class LoadCredentialsResponseDeserializer + extends JsonDeserializer { + @Override + public T deserialize(JsonParser p, DeserializationContext context) throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return (T) LoadCredentialsResponseParser.fromJson(jsonNode); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponse.java new file mode 100644 index 000000000000..410981291046 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponse.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.responses; + +import java.util.List; +import org.apache.iceberg.rest.RESTResponse; +import org.apache.iceberg.rest.credentials.Credential; +import org.immutables.value.Value; + +@Value.Immutable +public interface LoadCredentialsResponse extends RESTResponse { + List credentials(); + + @Override + default void validate() { + // nothing to validate + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponseParser.java new file mode 100644 index 000000000000..9ee0b9c35e1e --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadCredentialsResponseParser.java @@ -0,0 +1,77 @@ +/* + * 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.rest.responses; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.CredentialParser; +import org.apache.iceberg.util.JsonUtil; + +public class LoadCredentialsResponseParser { + private static final String STORAGE_CREDENTIALS = "storage-credentials"; + + private LoadCredentialsResponseParser() {} + + public static String toJson(LoadCredentialsResponse response) { + return toJson(response, false); + } + + public static String toJson(LoadCredentialsResponse response, boolean pretty) { + return JsonUtil.generate(gen -> toJson(response, gen), pretty); + } + + public static void toJson(LoadCredentialsResponse response, JsonGenerator gen) + throws IOException { + Preconditions.checkArgument(null != response, "Invalid load credentials response: null"); + + gen.writeStartObject(); + + gen.writeArrayFieldStart(STORAGE_CREDENTIALS); + for (Credential credential : response.credentials()) { + CredentialParser.toJson(credential, gen); + } + + gen.writeEndArray(); + + gen.writeEndObject(); + } + + public static LoadCredentialsResponse fromJson(String json) { + return JsonUtil.parse(json, LoadCredentialsResponseParser::fromJson); + } + + public static LoadCredentialsResponse fromJson(JsonNode json) { + Preconditions.checkArgument( + null != json, "Cannot parse load credentials response from null object"); + + JsonNode credentials = JsonUtil.get(STORAGE_CREDENTIALS, json); + Preconditions.checkArgument( + credentials.isArray(), "Cannot parse credentials from non-array: %s", credentials); + + ImmutableLoadCredentialsResponse.Builder builder = ImmutableLoadCredentialsResponse.builder(); + for (JsonNode credential : credentials) { + builder.addCredentials(CredentialParser.fromJson(credential)); + } + + return builder.build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java index 875403d703ab..8d34b1498369 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java @@ -98,13 +98,7 @@ public static LoadTableResponse fromJson(JsonNode json) { } if (json.hasNonNull(STORAGE_CREDENTIALS)) { - JsonNode credentials = JsonUtil.get(STORAGE_CREDENTIALS, json); - Preconditions.checkArgument( - credentials.isArray(), "Cannot parse credentials from non-array: %s", credentials); - - for (JsonNode credential : credentials) { - builder.addCredential(CredentialParser.fromJson(credential)); - } + builder.addAllCredentials(LoadCredentialsResponseParser.fromJson(json).credentials()); } return builder.build(); diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java index 61d8fce1dd51..aedf05cf62a9 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java @@ -93,13 +93,7 @@ public static LoadViewResponse fromJson(JsonNode json) { } if (json.hasNonNull(STORAGE_CREDENTIALS)) { - JsonNode credentials = JsonUtil.get(STORAGE_CREDENTIALS, json); - Preconditions.checkArgument( - credentials.isArray(), "Cannot parse credentials from non-array: %s", credentials); - - for (JsonNode credential : credentials) { - builder.addCredentials(CredentialParser.fromJson(credential)); - } + builder.addAllCredentials(LoadCredentialsResponseParser.fromJson(json).credentials()); } return builder.build(); diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadCredentialsResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadCredentialsResponseParser.java new file mode 100644 index 000000000000..f2e723da2540 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadCredentialsResponseParser.java @@ -0,0 +1,112 @@ +/* + * 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.rest.responses; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.credentials.ImmutableCredential; +import org.junit.jupiter.api.Test; + +public class TestLoadCredentialsResponseParser { + @Test + public void nullCheck() { + assertThatThrownBy(() -> LoadCredentialsResponseParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid load credentials response: null"); + + assertThatThrownBy(() -> LoadCredentialsResponseParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse load credentials response from null object"); + } + + @Test + public void missingFields() { + assertThatThrownBy(() -> LoadCredentialsResponseParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: storage-credentials"); + + assertThatThrownBy(() -> LoadCredentialsResponseParser.fromJson("{\"x\": \"val\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: storage-credentials"); + } + + @Test + public void roundTripSerde() { + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder() + .addCredentials( + ImmutableCredential.builder() + .prefix("s3://custom-uri") + .config( + ImmutableMap.of( + "s3.access-key-id", + "keyId", + "s3.secret-access-key", + "accessKey", + "s3.session-token", + "sessionToken")) + .build()) + .addCredentials( + ImmutableCredential.builder() + .prefix("gs://custom-uri") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken1", "gcs.oauth2.token-expires-at", "1000")) + .build()) + .addCredentials( + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken2", "gcs.oauth2.token-expires-at", "2000")) + .build()) + .build(); + + String expectedJson = + "{\n" + + " \"storage-credentials\" : [ {\n" + + " \"prefix\" : \"s3://custom-uri\",\n" + + " \"config\" : {\n" + + " \"s3.access-key-id\" : \"keyId\",\n" + + " \"s3.secret-access-key\" : \"accessKey\",\n" + + " \"s3.session-token\" : \"sessionToken\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken1\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken2\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"2000\"\n" + + " }\n" + + " } ]\n" + + "}"; + + String json = LoadCredentialsResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(LoadCredentialsResponseParser.fromJson(json)).isEqualTo(response); + } +} From 35a02d035e40344523fdb7a0933e07a8433ea763 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 24 Oct 2024 16:07:35 +0200 Subject: [PATCH 037/313] OpenAPI: Add endpoint for refreshing vended credentials (#11281) --- open-api/rest-catalog-open-api.py | 6 ++++ open-api/rest-catalog-open-api.yaml | 55 +++++++++++++++++++++++++++++ 2 files changed, 61 insertions(+) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index a16bf32155d3..e9cce361dd88 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -475,6 +475,12 @@ class StorageCredential(BaseModel): config: Dict[str, str] +class LoadCredentialsResponse(BaseModel): + storage_credentials: List[StorageCredential] = Field( + ..., alias='storage-credentials' + ) + + class PlanStatus(BaseModel): __root__: Literal['completed', 'submitted', 'cancelled', 'failed'] = Field( ..., description='Status of a server-side planning operation' diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 212a17c1295a..9835a96e0aa3 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -1159,6 +1159,44 @@ paths: 5XX: $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/namespaces/{namespace}/tables/{table}/credentials: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/table' + + get: + tags: + - Catalog API + summary: Load vended credentials for a table from the catalog + operationId: loadCredentials + description: Load vended credentials for a table from the catalog. + responses: + 200: + $ref: '#/components/responses/LoadCredentialsResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found - NoSuchTableException, table to load credentials for does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + TableToLoadDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/tables/rename: parameters: - $ref: '#/components/parameters/prefix' @@ -3118,6 +3156,16 @@ components: additionalProperties: type: string + LoadCredentialsResponse: + type: object + required: + - storage-credentials + properties: + storage-credentials: + type: array + items: + $ref: '#/components/schemas/StorageCredential' + LoadTableResult: description: | Result used when a table is successfully loaded. @@ -4579,6 +4627,13 @@ components: schema: $ref: '#/components/schemas/CommitTableResponse' + LoadCredentialsResponse: + description: Table credentials result when loading credentials for a table + content: + application/json: + schema: + $ref: '#/components/schemas/LoadCredentialsResponse' + ####################################### # Common examples of different values # ####################################### From 6ba1a1f578b2445f9bf3a2721fae538275bc9948 Mon Sep 17 00:00:00 2001 From: sullis Date: Thu, 24 Oct 2024 08:37:33 -0700 Subject: [PATCH 038/313] AWS: Use testcontainers-minio instead of S3Mock (#11349) --- .../apache/iceberg/aws/s3/MinioContainer.java | 68 ----------------- .../org/apache/iceberg/aws/s3/MinioUtil.java | 65 ++++++++++++++++ .../apache/iceberg/aws/s3/TestMinioUtil.java | 75 +++++++++++++++++++ .../apache/iceberg/aws/s3/TestS3FileIO.java | 13 ++-- .../iceberg/aws/s3/TestS3InputStream.java | 13 ++-- .../iceberg/aws/s3/TestS3OutputStream.java | 13 ++-- .../aws/s3/signer/S3SignerServlet.java | 6 +- .../aws/s3/signer/TestS3RestSigner.java | 18 ++++- build.gradle | 8 +- gradle/libs.versions.toml | 4 +- 10 files changed, 180 insertions(+), 103 deletions(-) delete mode 100644 aws/src/test/java/org/apache/iceberg/aws/s3/MinioContainer.java create mode 100644 aws/src/test/java/org/apache/iceberg/aws/s3/MinioUtil.java create mode 100644 aws/src/test/java/org/apache/iceberg/aws/s3/TestMinioUtil.java diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/MinioContainer.java b/aws/src/test/java/org/apache/iceberg/aws/s3/MinioContainer.java deleted file mode 100644 index b6a8d960981c..000000000000 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/MinioContainer.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.aws.s3; - -import java.net.URI; -import java.time.Duration; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; -import org.testcontainers.utility.Base58; -import software.amazon.awssdk.auth.credentials.AwsCredentials; - -public class MinioContainer extends GenericContainer { - - private static final int DEFAULT_PORT = 9000; - private static final String DEFAULT_IMAGE = "minio/minio"; - private static final String DEFAULT_TAG = "edge"; - - private static final String MINIO_ACCESS_KEY = "MINIO_ACCESS_KEY"; - private static final String MINIO_SECRET_KEY = "MINIO_SECRET_KEY"; - - private static final String DEFAULT_STORAGE_DIRECTORY = "/data"; - private static final String HEALTH_ENDPOINT = "/minio/health/ready"; - - public MinioContainer(AwsCredentials credentials) { - this(DEFAULT_IMAGE + ":" + DEFAULT_TAG, credentials); - } - - public MinioContainer(String image, AwsCredentials credentials) { - super(image == null ? DEFAULT_IMAGE + ":" + DEFAULT_TAG : image); - this.withNetworkAliases("minio-" + Base58.randomString(6)) - .withCommand("server", DEFAULT_STORAGE_DIRECTORY) - .addExposedPort(DEFAULT_PORT); - if (credentials != null) { - this.withEnv(MINIO_ACCESS_KEY, credentials.accessKeyId()) - .withEnv(MINIO_SECRET_KEY, credentials.secretAccessKey()); - } - - // this enables virtual-host-style requests. see - // https://github.com/minio/minio/tree/master/docs/config#domain - this.withEnv("MINIO_DOMAIN", "localhost"); - - setWaitStrategy( - new HttpWaitStrategy() - .forPort(DEFAULT_PORT) - .forPath(HEALTH_ENDPOINT) - .withStartupTimeout(Duration.ofMinutes(2))); - } - - public URI getURI() { - return URI.create("http://" + getHost() + ":" + getMappedPort(DEFAULT_PORT)); - } -} diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/MinioUtil.java b/aws/src/test/java/org/apache/iceberg/aws/s3/MinioUtil.java new file mode 100644 index 000000000000..ff131784a55b --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/MinioUtil.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.aws.s3; + +import java.net.URI; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.utility.DockerImageName; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.S3ClientBuilder; + +public class MinioUtil { + + private MinioUtil() {} + + public static MinIOContainer createContainer() { + return createContainer(null); + } + + public static MinIOContainer createContainer(AwsCredentials credentials) { + var container = new MinIOContainer(DockerImageName.parse("minio/minio:latest")); + + // this enables virtual-host-style requests. see + // https://github.com/minio/minio/tree/master/docs/config#domain + container.withEnv("MINIO_DOMAIN", "localhost"); + + if (credentials != null) { + container.withUserName(credentials.accessKeyId()); + container.withPassword(credentials.secretAccessKey()); + } + + return container; + } + + public static S3Client createS3Client(MinIOContainer container) { + URI uri = URI.create(container.getS3URL()); + S3ClientBuilder builder = S3Client.builder(); + builder.credentialsProvider( + StaticCredentialsProvider.create( + AwsBasicCredentials.create(container.getUserName(), container.getPassword()))); + builder.applyMutation(mutator -> mutator.endpointOverride(uri)); + builder.region(Region.US_EAST_1); + builder.forcePathStyle(true); // OSX won't resolve subdomains + return builder.build(); + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestMinioUtil.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestMinioUtil.java new file mode 100644 index 000000000000..9955aa7f8459 --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestMinioUtil.java @@ -0,0 +1,75 @@ +/* + * 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.aws.s3; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.UUID; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.core.sync.ResponseTransformer; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.CreateBucketResponse; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; +import software.amazon.awssdk.services.s3.model.S3Exception; + +@Testcontainers +public class TestMinioUtil { + @Container private static final MinIOContainer MINIO = MinioUtil.createContainer(); + + @Test + void validateS3ConditionalWrites() { + S3Client s3Client = MinioUtil.createS3Client(MINIO); + + String bucket = "test-bucket-" + UUID.randomUUID(); + + CreateBucketResponse createBucketResponse = + s3Client.createBucket(CreateBucketRequest.builder().bucket(bucket).build()); + assertThat(createBucketResponse.sdkHttpResponse().isSuccessful()).isTrue(); + + String key = "test-key-" + UUID.randomUUID().toString(); + for (int i = 0; i < 5; i++) { + String payload = "test-payload-" + i; + PutObjectRequest request = + PutObjectRequest.builder().bucket(bucket).key(key).ifNoneMatch("*").build(); + RequestBody body = RequestBody.fromString(payload); + if (i == 0) { + PutObjectResponse response = s3Client.putObject(request, body); + assertThat(response.sdkHttpResponse().isSuccessful()).isTrue(); + } else { + assertThatThrownBy(() -> s3Client.putObject(request, body)) + .isInstanceOf(S3Exception.class) + .hasMessageContaining("Service: S3, Status Code: 412") + .hasMessageContaining("At least one of the pre-conditions you specified did not hold"); + } + } + + var getResponse = + s3Client.getObject( + request -> request.bucket(bucket).key(key), ResponseTransformer.toBytes()); + String responseBody = getResponse.asUtf8String(); + assertThat(responseBody).isEqualTo("test-payload-0"); + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java index 77717d796176..cda6216fe83c 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java @@ -31,7 +31,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import com.adobe.testing.s3mock.junit5.S3MockExtension; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -84,9 +83,10 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; import org.mockito.Mockito; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; @@ -103,12 +103,11 @@ import software.amazon.awssdk.services.s3.model.S3Object; import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Iterable; -@ExtendWith(S3MockExtension.class) +@Testcontainers public class TestS3FileIO { - @RegisterExtension - public static final S3MockExtension S3_MOCK = S3MockExtension.builder().silent().build(); + @Container private static final MinIOContainer MINIO = MinioUtil.createContainer(); - public SerializableSupplier s3 = S3_MOCK::createS3ClientV2; + private final SerializableSupplier s3 = () -> MinioUtil.createS3Client(MINIO); private final S3Client s3mock = mock(S3Client.class, delegatesTo(s3.get())); private final Random random = new Random(1); private final int numBucketsForBatchDeletion = 3; diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java index f5b78eddaaad..f8903842df37 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java @@ -21,7 +21,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import com.adobe.testing.s3mock.junit5.S3MockExtension; import java.io.IOException; import java.util.Arrays; import java.util.Random; @@ -30,8 +29,9 @@ import org.apache.iceberg.io.SeekableInputStream; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.BucketAlreadyExistsException; @@ -39,12 +39,11 @@ import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.PutObjectRequest; -@ExtendWith(S3MockExtension.class) +@Testcontainers public class TestS3InputStream { - @RegisterExtension - public static final S3MockExtension S3_MOCK = S3MockExtension.builder().silent().build(); + @Container private static final MinIOContainer MINIO = MinioUtil.createContainer(); - private final S3Client s3 = S3_MOCK.createS3ClientV2(); + private final S3Client s3 = MinioUtil.createS3Client(MINIO); private final Random random = new Random(1); @BeforeEach diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java index 88488bf4c313..ab173d06992e 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java @@ -31,7 +31,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import com.adobe.testing.s3mock.junit5.S3MockExtension; import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -50,12 +49,13 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.core.ResponseBytes; import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.core.sync.ResponseTransformer; @@ -72,16 +72,15 @@ import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.utils.BinaryUtils; -@ExtendWith(S3MockExtension.class) +@Testcontainers public class TestS3OutputStream { private static final Logger LOG = LoggerFactory.getLogger(TestS3OutputStream.class); private static final String BUCKET = "test-bucket"; private static final int FIVE_MBS = 5 * 1024 * 1024; - @RegisterExtension - public static final S3MockExtension S3_MOCK = S3MockExtension.builder().silent().build(); + @Container private static final MinIOContainer MINIO = MinioUtil.createContainer(); - private final S3Client s3 = S3_MOCK.createS3ClientV2(); + private final S3Client s3 = MinioUtil.createS3Client(MINIO); private final S3Client s3mock = mock(S3Client.class, delegatesTo(s3)); private final Random random = new Random(1); private final Path tmpDir = Files.createTempDirectory("s3fileio-test-"); diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java index ce7527af765c..038d76b03e4b 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java @@ -24,6 +24,9 @@ import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import com.fasterxml.jackson.databind.ObjectMapper; +import jakarta.servlet.http.HttpServlet; +import jakarta.servlet.http.HttpServletRequest; +import jakarta.servlet.http.HttpServletResponse; import java.io.InputStreamReader; import java.io.Reader; import java.time.Clock; @@ -37,9 +40,6 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; -import javax.servlet.http.HttpServlet; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; import org.apache.hc.core5.http.ContentType; import org.apache.hc.core5.http.HttpHeaders; import org.apache.iceberg.exceptions.RESTException; diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java index f2a70aee29e5..313214c4e98f 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java @@ -20,6 +20,7 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.net.URI; import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; @@ -27,7 +28,7 @@ import java.util.Map; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.stream.Collectors; -import org.apache.iceberg.aws.s3.MinioContainer; +import org.apache.iceberg.aws.s3.MinioUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.rest.auth.OAuth2Properties; @@ -41,6 +42,9 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.testcontainers.containers.MinIOContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; @@ -67,15 +71,19 @@ import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.UploadPartRequest; +@Testcontainers public class TestS3RestSigner { private static final Region REGION = Region.US_WEST_2; private static final String BUCKET = "iceberg-s3-signer-test"; + static final AwsCredentialsProvider CREDENTIALS_PROVIDER = StaticCredentialsProvider.create( AwsBasicCredentials.create("accessKeyId", "secretAccessKey")); - private static final MinioContainer MINIO_CONTAINER = - new MinioContainer(CREDENTIALS_PROVIDER.resolveCredentials()); + + @Container + private static final MinIOContainer MINIO_CONTAINER = + MinioUtil.createContainer(CREDENTIALS_PROVIDER.resolveCredentials()); private static Server httpServer; private static ValidatingSigner validatingSigner; @@ -83,6 +91,8 @@ public class TestS3RestSigner { @BeforeAll public static void beforeClass() throws Exception { + assertThat(MINIO_CONTAINER.isRunning()).isTrue(); + if (null == httpServer) { httpServer = initHttpServer(); } @@ -141,7 +151,7 @@ public void before() throws Exception { s3ClientBuilder -> s3ClientBuilder.httpClientBuilder( software.amazon.awssdk.http.apache.ApacheHttpClient.builder())) - .endpointOverride(MINIO_CONTAINER.getURI()) + .endpointOverride(URI.create(MINIO_CONTAINER.getS3URL())) .forcePathStyle(true) // OSX won't resolve subdomains .overrideConfiguration( c -> c.putAdvancedOption(SdkAdvancedClientOption.SIGNER, validatingSigner)) diff --git a/build.gradle b/build.gradle index 9e4b62d87576..a654e1cba5fb 100644 --- a/build.gradle +++ b/build.gradle @@ -495,20 +495,18 @@ project(':iceberg-aws') { testImplementation("software.amazon.awssdk:s3control") testImplementation("software.amazon.s3.accessgrants:aws-s3-accessgrants-java-plugin") testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation(libs.s3mock.junit5) { - exclude module: "spring-boot-starter-logging" - exclude module: "logback-classic" - exclude group: 'junit' - } testImplementation libs.esotericsoftware.kryo testImplementation libs.sqlite.jdbc testImplementation libs.testcontainers + testImplementation libs.testcontainers.junit.jupiter + testImplementation libs.testcontainers.minio testImplementation libs.httpcomponents.httpclient5 testImplementation libs.mockserver.netty testImplementation libs.mockserver.client.java testImplementation libs.jaxb.api testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation libs.awaitility + testImplementation libs.jetty.servlet } sourceSets { diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index deb06003212c..2a01ea0b99f8 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -78,7 +78,6 @@ orc = "1.9.4" parquet = "1.14.3" pig = "0.17.0" roaringbitmap = "1.3.0" -s3mock-junit5 = "2.17.0" scala-collection-compat = "2.12.0" slf4j = "2.0.16" snowflake-jdbc = "3.19.0" @@ -220,12 +219,13 @@ nessie-jaxrs-testextension = { module = "org.projectnessie.nessie:nessie-jaxrs-t nessie-versioned-storage-inmemory-tests = { module = "org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests", version.ref = "nessie" } nessie-versioned-storage-testextension = { module = "org.projectnessie.nessie:nessie-versioned-storage-testextension", version.ref = "nessie" } orc-tools = { module = "org.apache.orc:orc-tools", version.ref = "orc" } -s3mock-junit5 = { module = "com.adobe.testing:s3mock-junit5", version.ref = "s3mock-junit5" } spring-boot-starter-jetty = { module = "org.springframework.boot:spring-boot-starter-jetty", version.ref = "spring-boot" } spring-boot-starter-web = { module = "org.springframework.boot:spring-boot-starter-web", version.ref = "spring-boot" } spring-web = { module = "org.springframework:spring-web", version.ref = "spring-web" } sqlite-jdbc = { module = "org.xerial:sqlite-jdbc", version.ref = "sqlite-jdbc" } testcontainers = { module = "org.testcontainers:testcontainers", version.ref = "testcontainers" } +testcontainers-junit-jupiter = { module = "org.testcontainers:junit-jupiter", version.ref = "testcontainers" } +testcontainers-minio = { module = "org.testcontainers:minio", version.ref = "testcontainers" } tez010-dag = { module = "org.apache.tez:tez-dag", version.ref = "tez010" } tez010-mapreduce = { module = "org.apache.tez:tez-mapreduce", version.ref = "tez010" } tez08-dag = { module = "org.apache.tez:tez-dag", version.ref = "tez08" } From 12ac3ee5d358521a52318d276592ba9cf95e8926 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Thu, 24 Oct 2024 10:58:56 -0700 Subject: [PATCH 039/313] Kafka Connect: Include third party licenses and notices in distribution (#10829) * Include third party licenses and notices in distribution * update licenses and notices * remove kite notice * Revert "remove kite notice" This reverts commit 16c01737d2f30d152c07089c91ebfa3fe9ec1d6c. * Only include notices and licenses for direct deps of Iceberg modules * Revert "Only include notices and licenses for direct deps of Iceberg modules" This reverts commit 54245e7dfcf4b1972d6a710fa48b7123274d6906. --- kafka-connect/build.gradle | 6 +- kafka-connect/kafka-connect-runtime/LICENSE | 1970 +++++++++++++++++++ kafka-connect/kafka-connect-runtime/NOTICE | 1723 ++++++++++++++++ 3 files changed, 3697 insertions(+), 2 deletions(-) create mode 100644 kafka-connect/kafka-connect-runtime/LICENSE create mode 100644 kafka-connect/kafka-connect-runtime/NOTICE diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index bef660fb6580..d38d01768e66 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -198,7 +198,8 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { from configurations.runtimeClasspath } into('doc/') { - from "$rootDir/LICENSE" + from "$projectDir/LICENSE" + from "$projectDir/NOTICE" } into('assets/') { from "${processResources.destinationDir}/iceberg.png" @@ -212,7 +213,8 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { from configurations.hive } into('doc/') { - from "$rootDir/LICENSE" + from "$projectDir/LICENSE" + from "$projectDir/NOTICE" } into('assets/') { from "${processResources.destinationDir}/iceberg.png" diff --git a/kafka-connect/kafka-connect-runtime/LICENSE b/kafka-connect/kafka-connect-runtime/LICENSE new file mode 100644 index 000000000000..6eb4fdee65b8 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/LICENSE @@ -0,0 +1,1970 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. + +-------------------------------------------------------------------------------- + +This binary artifact contains code from the following projects: + +-------------------------------------------------------------------------------- + +Group: aopalliance Name: aopalliance Version: 1.0 +Project URL (from POM): http://aopalliance.sourceforge.net +License (from POM): Public Domain +-------------------------------------------------------------------------------- + +Group: ch.qos.reload4j Name: reload4j Version: 1.2.22 +Project URL (from manifest): https://reload4j.qos.ch/ +Project URL (from POM): https://reload4j.qos.ch +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-core Version: 1.49.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-core-http-netty Version: 1.15.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-identity Version: 1.13.0 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-json Version: 1.1.0 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-storage-blob Version: 12.26.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-storage-common Version: 12.25.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-storage-file-datalake Version: 12.19.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-storage-internal-avro Version: 12.11.1 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.azure Name: azure-xml Version: 1.0.0 +Project URL (from POM): https://github.com/Azure/azure-sdk-for-java +License (from POM): The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-annotations Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson +Project URL (from POM): https://github.com/FasterXML/jackson +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-core Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-core +Project URL (from POM): https://github.com/FasterXML/jackson-core +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-databind Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson +Project URL (from POM): https://github.com/FasterXML/jackson +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.dataformat Name: jackson-dataformat-xml Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-dataformat-xml +Project URL (from POM): https://github.com/FasterXML/jackson-dataformat-xml +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.datatype Name: jackson-datatype-jsr310 Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310 +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.jaxrs Name: jackson-jaxrs-base Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-jaxrs-providers/jackson-jaxrs-base +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.jaxrs Name: jackson-jaxrs-json-provider Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-jaxrs-providers/jackson-jaxrs-json-provider +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.module Name: jackson-module-jaxb-annotations Version: 2.17.2 +Project URL (from manifest): https://github.com/FasterXML/jackson-modules-base +Project URL (from POM): https://github.com/FasterXML/jackson-modules-base +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.woodstox Name: woodstox-core Version: 6.7.0 +Project URL (from manifest): https://github.com/FasterXML/woodstox +Project URL (from POM): https://github.com/FasterXML/woodstox +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.github.ben-manes.caffeine Name: caffeine Version: 2.9.3 +Project URL (from POM): https://github.com/ben-manes/caffeine +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.github.luben Name: zstd-jni Version: 1.5.0-1 +License URL (from manifest): https://opensource.org/licenses/BSD-2-Clause +Project URL (from POM): https://github.com/luben/zstd-jni +License (from POM): BSD 2-Clause License - https://opensource.org/licenses/BSD-2-Clause + +-------------------------------------------------------------------------------- + +Group: com.github.pjfanning Name: jersey-json Version: 1.20 +Project URL (from POM): https://github.com/pjfanning/jersey-1.x +License (from POM): CDDL 1.1 - http://glassfish.java.net/public/CDDL+GPL_1_1.html +License (from POM): GPL2 w/ CPE - http://glassfish.java.net/public/CDDL+GPL_1_1.html + +-------------------------------------------------------------------------------- + +Group: com.github.stephenc.jcip Name: jcip-annotations Version: 1.0-1 +Project URL (from POM): http://stephenc.github.com/jcip-annotations +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.android Name: annotations Version: 4.1.1.4 +Project URL (from POM): http://source.android.com/ +License (from POM): Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: api-common Version: 2.33.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/api-common-java/blob/main/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: gax Version: 2.50.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/gax-java/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: gax-grpc Version: 2.50.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/gax-java/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: gax-httpjson Version: 2.50.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/gax-java/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api-client Name: google-api-client Version: 2.6.0 +Project URL (from manifest): https://developers.google.com/api-client-library/java/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: gapic-google-cloud-storage-v2 Version: 2.40.1-alpha +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: grpc-google-cloud-storage-v2 Version: 2.40.1-alpha +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: proto-google-cloud-storage-v2 Version: 2.40.1-alpha +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: proto-google-common-protos Version: 2.41.0 +Project URL (from POM): https://github.com/googleapis/sdk-platform-java +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: proto-google-iam-v1 Version: 1.36.0 +Project URL (from POM): https://github.com/googleapis/sdk-platform-java +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.apis Name: google-api-services-storage Version: v1-rev20240621-2.0.0 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.auth Name: google-auth-library-credentials Version: 1.23.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD New license - http://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.auth Name: google-auth-library-oauth2-http Version: 1.23.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD New license - http://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.auto.value Name: auto-value-annotations Version: 1.10.4 +Project URL (from POM): https://github.com/google/auto/tree/main/value +License (from POM): Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-core Version: 2.40.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-core-grpc Version: 2.40.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-core-http Version: 2.40.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-storage Version: 2.40.1 +Project URL (from POM): https://github.com/googleapis/java-storage +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.code.findbugs Name: jsr305 Version: 3.0.2 +Project URL (from POM): http://findbugs.sourceforge.net/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.code.gson Name: gson Version: 2.11.0 +Project URL (from manifest): https://github.com/google/gson +Manifest License: "Apache-2.0";link="https://www.apache.org/licenses/LICENSE-2.0.txt" (Not packaged) +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.errorprone Name: error_prone_annotations Version: 2.28.0 +Project URL (from manifest): https://errorprone.info/error_prone_annotations +Manifest License: "Apache 2.0";link="http://www.apache.org/licenses/LICENSE-2.0.txt" (Not packaged) +License (from POM): Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.guava Name: failureaccess Version: 1.0.2 +Project URL (from manifest): https://github.com/google/guava/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.guava Name: guava Version: 33.1.0-jre +Project URL (from manifest): https://github.com/google/guava/ +Project URL (from POM): https://github.com/google/guava +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.guava Name: listenablefuture Version: 9999.0-empty-to-avoid-conflict-with-guava +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client Version: 1.44.2 +Project URL (from manifest): https://www.google.com/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client-apache-v2 Version: 1.44.2 +Project URL (from manifest): https://www.google.com/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client-appengine Version: 1.44.2 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client-gson Version: 1.44.2 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.http-client Name: google-http-client-jackson2 Version: 1.44.2 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.inject Name: guice Version: 4.0 +Project URL (from manifest): https://github.com/google/guice +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.j2objc Name: j2objc-annotations Version: 3.0.0 +Project URL (from POM): https://github.com/google/j2objc/ +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.oauth-client Name: google-oauth-client Version: 1.36.0 +Project URL (from manifest): https://www.google.com/ +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.protobuf Name: protobuf-java Version: 3.25.3 +Project URL (from manifest): https://developers.google.com/protocol-buffers/ +License (from POM): BSD-3-Clause - https://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.protobuf Name: protobuf-java-util Version: 3.25.3 +Project URL (from manifest): https://developers.google.com/protocol-buffers/ +License (from POM): BSD-3-Clause - https://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.re2j Name: re2j Version: 1.7 +Project URL (from POM): http://github.com/google/re2j +License (from POM): Go License - https://golang.org/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.jcraft Name: jsch Version: 0.1.55 +Project URL (from POM): http://www.jcraft.com/jsch/ +License (from POM): Revised BSD - http://www.jcraft.com/jsch/LICENSE.txt + +-------------------------------------------------------------------------------- + +Group: com.jolbox Name: bonecp Version: 0.8.0.RELEASE +Project URL (from manifest): http://jolbox.com +License (from POM): Apache v2 - http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: com.microsoft.azure Name: msal4j Version: 1.15.1 +Project URL (from manifest): https://github.com/AzureAD/microsoft-authentication-library-for-java +Manifest License: "MIT License" (Not packaged) +Project URL (from POM): https://github.com/AzureAD/microsoft-authentication-library-for-java +License (from POM): MIT License +-------------------------------------------------------------------------------- + +Group: com.microsoft.azure Name: msal4j-persistence-extension Version: 1.3.0 +Project URL (from POM): https://github.com/AzureAD/microsoft-authentication-library-for-java +License (from POM): MIT License +-------------------------------------------------------------------------------- + +Group: com.nimbusds Name: content-type Version: 2.3 +Project URL (from manifest): https://connect2id.com +Project URL (from POM): https://bitbucket.org/connect2id/nimbus-content-type +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.nimbusds Name: lang-tag Version: 1.7 +Project URL (from manifest): https://connect2id.com/ +Project URL (from POM): https://bitbucket.org/connect2id/nimbus-language-tags +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.nimbusds Name: nimbus-jose-jwt Version: 9.37.3 +Project URL (from manifest): https://connect2id.com +Project URL (from POM): https://bitbucket.org/connect2id/nimbus-jose-jwt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.nimbusds Name: oauth2-oidc-sdk Version: 11.9.1 +Project URL (from manifest): https://bitbucket.org/connect2id/oauth-2.0-sdk-with-openid-connect-extensions +Manifest License: "Apache License, version 2.0";link="https://www.apache.org/licenses/LICENSE-2.0.html" (Not packaged) +Project URL (from POM): https://bitbucket.org/connect2id/oauth-2.0-sdk-with-openid-connect-extensions +License (from POM): Apache License, version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: com.squareup.okhttp3 Name: okhttp Version: 4.9.3 +Project URL (from POM): https://square.github.io/okhttp/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.squareup.okio Name: okio Version: 2.8.0 +Project URL (from POM): https://github.com/square/okio/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.sun.xml.bind Name: jaxb-impl Version: 2.2.3-1 +Project URL (from POM): http://jaxb.java.net/ +License (from POM): CDDL 1.1 - https://glassfish.java.net/public/CDDL+GPL_1_1.html +License (from POM): GPL2 w/ CPE - https://glassfish.java.net/public/CDDL+GPL_1_1.html + +-------------------------------------------------------------------------------- + +Group: commons-beanutils Name: commons-beanutils Version: 1.9.4 +Project URL (from manifest): https://commons.apache.org/proper/commons-beanutils/ +Project URL (from POM): https://commons.apache.org/proper/commons-beanutils/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-cli Name: commons-cli Version: 1.2 +Project URL (from manifest): http://commons.apache.org/cli/ +Project URL (from POM): http://commons.apache.org/cli/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-codec Name: commons-codec Version: 1.17.1 +Project URL (from manifest): https://commons.apache.org/proper/commons-codec/ +Project URL (from POM): https://commons.apache.org/proper/commons-codec/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-collections Name: commons-collections Version: 3.2.2 +Project URL (from manifest): http://commons.apache.org/collections/ +Project URL (from POM): http://commons.apache.org/collections/ +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-dbcp Name: commons-dbcp Version: 1.4 +Project URL (from manifest): http://commons.apache.org/dbcp/ +Project URL (from POM): http://commons.apache.org/dbcp/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-io Name: commons-io Version: 2.16.1 +Project URL (from manifest): https://commons.apache.org/proper/commons-io/ +Project URL (from POM): https://commons.apache.org/proper/commons-io/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-lang Name: commons-lang Version: 2.6 +Project URL (from manifest): http://commons.apache.org/lang/ +Project URL (from POM): http://commons.apache.org/lang/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-logging Name: commons-logging Version: 1.2 +Project URL (from manifest): http://commons.apache.org/proper/commons-logging/ +Project URL (from POM): http://commons.apache.org/proper/commons-logging/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-net Name: commons-net Version: 3.9.0 +Project URL (from manifest): https://commons.apache.org/proper/commons-net/ +Project URL (from POM): https://commons.apache.org/proper/commons-net/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-pool Name: commons-pool Version: 1.6 +Project URL (from manifest): http://commons.apache.org/pool/ +Project URL (from POM): http://commons.apache.org/pool/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: dnsjava Name: dnsjava Version: 2.1.7 +Project URL (from POM): http://www.dnsjava.org +License (from POM): BSD 2-Clause license - http://opensource.org/licenses/BSD-2-Clause + +-------------------------------------------------------------------------------- + +Group: io.airlift Name: aircompressor Version: 0.27 +Project URL (from POM): https://github.com/airlift/aircompressor +License (from POM): Apache License 2.0 - https://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: io.dropwizard.metrics Name: metrics-core Version: 3.2.4 +License (from POM): Apache License 2.0 - http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: io.dropwizard.metrics Name: metrics-json Version: 3.1.0 +License (from POM): Apache License 2.0 - http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: io.dropwizard.metrics Name: metrics-jvm Version: 3.1.0 +License (from POM): Apache License 2.0 - http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-alts Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-api Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-auth Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-context Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-core Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-googleapis Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-grpclb Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-inprocess Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-netty-shaded Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-protobuf Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-protobuf-lite Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-rls Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-services Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-stub Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-util Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-xds Version: 1.62.2 +Project URL (from POM): https://github.com/grpc/grpc-java +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-buffer Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec-dns Version: 4.1.109.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec-http Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec-http2 Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-codec-socks Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-common Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-handler Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-handler-proxy Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-resolver Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-resolver-dns Version: 4.1.109.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-resolver-dns-classes-macos Version: 4.1.109.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-resolver-dns-native-macos Version: 4.1.109.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-tcnative-boringssl-static Version: 2.0.65.Final +Project URL (from manifest): https://netty.io/ +Project URL (from POM): https://github.com/netty/netty-tcnative/netty-tcnative-boringssl-static/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-tcnative-classes Version: 2.0.65.Final +Project URL (from manifest): https://netty.io/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-classes-epoll Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-classes-kqueue Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-native-epoll Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-native-kqueue Version: 4.1.110.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.netty Name: netty-transport-native-unix-common Version: 4.1.111.Final +Project URL (from manifest): https://netty.io/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.opencensus Name: opencensus-api Version: 0.31.1 +Project URL (from POM): https://github.com/census-instrumentation/opencensus-java +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.opencensus Name: opencensus-contrib-http-util Version: 0.31.1 +Project URL (from POM): https://github.com/census-instrumentation/opencensus-java +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.opencensus Name: opencensus-proto Version: 0.2.0 +Project URL (from POM): https://github.com/census-instrumentation/opencensus-proto +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.perfmark Name: perfmark-api Version: 0.27.0 +Project URL (from POM): https://github.com/perfmark/perfmark +License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 + +-------------------------------------------------------------------------------- + +Group: io.projectreactor Name: reactor-core Version: 3.4.38 +Project URL (from POM): https://github.com/reactor/reactor-core +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.projectreactor.netty Name: reactor-netty-core Version: 1.0.45 +Project URL (from POM): https://github.com/reactor/reactor-netty +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: io.projectreactor.netty Name: reactor-netty-http Version: 1.0.45 +Project URL (from POM): https://github.com/reactor/reactor-netty +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: jakarta.activation Name: jakarta.activation-api Version: 1.2.1 +Project URL (from manifest): https://www.eclipse.org +License (from POM): EDL 1.0 - http://www.eclipse.org/org/documents/edl-v10.php +License (from POM): Eclipse Public License v. 2.0 - https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt +License (from POM): GNU General Public License, version 2 with the GNU Classpath Exception - https://www.gnu.org/software/classpath/license.html + +-------------------------------------------------------------------------------- + +Group: jakarta.activation Name: jakarta.activation-api Version: 1.2.2 +Project URL (from manifest): https://www.eclipse.org +License (from POM): EDL 1.0 - http://www.eclipse.org/org/documents/edl-v10.php +License (from POM): Eclipse Public License v. 2.0 - https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt +License (from POM): GNU General Public License, version 2 with the GNU Classpath Exception - https://www.gnu.org/software/classpath/license.html + +-------------------------------------------------------------------------------- + +Group: jakarta.xml.bind Name: jakarta.xml.bind-api Version: 2.3.3 +Project URL (from manifest): https://www.eclipse.org +License (from POM): Eclipse Distribution License - v 1.0 - http://www.eclipse.org/org/documents/edl-v10.php +License (from POM): Eclipse Public License v. 2.0 - https://www.eclipse.org/org/documents/epl-2.0/EPL-2.0.txt +License (from POM): GNU General Public License, version 2 with the GNU Classpath Exception - https://www.gnu.org/software/classpath/license.html + +-------------------------------------------------------------------------------- + +Group: javax.annotation Name: javax.annotation-api Version: 1.3.2 +Project URL (from manifest): https://javaee.github.io/glassfish +Project URL (from POM): http://jcp.org/en/jsr/detail?id=250 +License (from POM): CDDL + GPLv2 with classpath exception - https://github.com/javaee/javax.annotation/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: javax.inject Name: javax.inject Version: 1 +Project URL (from POM): http://code.google.com/p/atinject/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: javax.jdo Name: jdo-api Version: 3.0.1 +Project URL (from POM): http://db.apache.org/jdo +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: javax.servlet Name: javax.servlet-api Version: 3.1.0 +Project URL (from manifest): https://glassfish.dev.java.net +Project URL (from POM): http://servlet-spec.java.net +License (from POM): CDDL + GPLv2 with classpath exception - https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + +-------------------------------------------------------------------------------- + +Group: javax.servlet.jsp Name: jsp-api Version: 2.1 + +-------------------------------------------------------------------------------- + +Group: javax.transaction Name: jta Version: 1.1 +Project URL (from POM): http://java.sun.com/products/jta + +-------------------------------------------------------------------------------- + +Group: javax.xml.bind Name: jaxb-api Version: 2.2.11 +Project URL (from manifest): http://www.oracle.com/ +Project URL (from POM): http://jaxb.java.net/ +License (from POM): CDDL 1.1 - https://glassfish.java.net/public/CDDL+GPL_1_1.html +License (from POM): GPL2 w/ CPE - https://glassfish.java.net/public/CDDL+GPL_1_1.html + +-------------------------------------------------------------------------------- + +Group: javax.xml.bind Name: jaxb-api Version: 2.2.2 +Project URL (from POM): https://jaxb.dev.java.net/ +License (from POM): CDDL 1.1 - https://glassfish.dev.java.net/public/CDDL+GPL_1_1.html +License (from POM): GPL2 w/ CPE - https://glassfish.dev.java.net/public/CDDL+GPL_1_1.html + +-------------------------------------------------------------------------------- + +Group: javax.xml.stream Name: stax-api Version: 1.0-2 +License (from POM): COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 - http://www.sun.com/cddl/cddl.html +License (from POM): GNU General Public Library - http://www.gnu.org/licenses/gpl.txt + +-------------------------------------------------------------------------------- + +Group: javolution Name: javolution Version: 5.5.1 +Project URL (from manifest): http://javolution.org +Project URL (from POM): http://javolution.org +License (from POM): BSD License - http://javolution.org/LICENSE.txt + +-------------------------------------------------------------------------------- + +Group: jline Name: jline Version: 2.12 +License (from POM): The BSD License - http://www.opensource.org/licenses/bsd-license.php + +-------------------------------------------------------------------------------- + +Group: joda-time Name: joda-time Version: 2.8.1 +Project URL (from manifest): http://www.joda.org/joda-time/ +Manifest License: Apache 2.0 (Not packaged) +Project URL (from POM): http://www.joda.org/joda-time/ +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: log4j Name: log4j Version: 1.2.17 +Project URL (from manifest): http://logging.apache.org/log4j/1.2 +Project URL (from POM): http://logging.apache.org/log4j/1.2/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: net.java.dev.jna Name: jna Version: 5.13.0 +Project URL (from POM): https://github.com/java-native-access/jna +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): LGPL-2.1-or-later - https://www.gnu.org/licenses/old-licenses/lgpl-2.1 + +-------------------------------------------------------------------------------- + +Group: net.java.dev.jna Name: jna-platform Version: 5.13.0 +Project URL (from POM): https://github.com/java-native-access/jna +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): LGPL-2.1-or-later - https://www.gnu.org/licenses/old-licenses/lgpl-2.1 + +-------------------------------------------------------------------------------- + +Group: net.minidev Name: accessors-smart Version: 2.5.0 +Project URL (from manifest): https://urielch.github.io/ +Project URL (from POM): https://urielch.github.io/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: net.minidev Name: json-smart Version: 2.5.0 +Project URL (from manifest): https://urielch.github.io/ +Project URL (from POM): https://urielch.github.io/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: net.sf.opencsv Name: opencsv Version: 2.3 +Project URL (from POM): http://opencsv.sf.net +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.antlr Name: antlr-runtime Version: 3.5.2 +Project URL (from POM): http://www.antlr.org +License (from POM): BSD licence - http://antlr.org/license.html + +-------------------------------------------------------------------------------- + +Group: org.apache.ant Name: ant Version: 1.9.1 +Project URL (from POM): http://ant.apache.org/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.ant Name: ant-launcher Version: 1.9.1 +Project URL (from POM): http://ant.apache.org/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.avro Name: avro Version: 1.12.0 +Project URL (from manifest): https://www.apache.org/ +Project URL (from POM): https://avro.apache.org +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-compress Version: 1.26.2 +Project URL (from manifest): https://commons.apache.org/proper/commons-compress/ +Project URL (from POM): https://commons.apache.org/proper/commons-compress/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-configuration2 Version: 2.8.0 +Project URL (from manifest): https://commons.apache.org/proper/commons-configuration/ +Project URL (from POM): https://commons.apache.org/proper/commons-configuration/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-lang3 Version: 3.14.0 +Project URL (from manifest): https://commons.apache.org/proper/commons-lang/ +Project URL (from POM): https://commons.apache.org/proper/commons-lang/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-math3 Version: 3.1.1 +Project URL (from manifest): http://commons.apache.org/math/ +Project URL (from POM): http://commons.apache.org/math/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-text Version: 1.10.0 +Project URL (from manifest): https://commons.apache.org/proper/commons-text +Project URL (from POM): https://commons.apache.org/proper/commons-text +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-client Version: 5.2.0 +Project URL (from manifest): http://www.apache.org/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-framework Version: 5.2.0 +Project URL (from manifest): http://www.apache.org/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-recipes Version: 5.2.0 +Project URL (from manifest): http://www.apache.org/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.derby Name: derby Version: 10.10.2.0 +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-annotations Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-auth Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-client Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-common Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-hdfs-client Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-mapreduce-client-common Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-mapreduce-client-core Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-mapreduce-client-jobclient Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-yarn-api Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-yarn-client Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-yarn-common Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop.thirdparty Name: hadoop-shaded-guava Version: 1.2.0 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop.thirdparty Name: hadoop-shaded-protobuf_3_7 Version: 1.1.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-common Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-metastore Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-serde Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-shims Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-storage-api Version: 2.4.0 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-0.23 Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-common Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-scheduler Version: 2.3.9 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents Name: httpclient Version: 4.5.13 +Project URL (from POM): http://hc.apache.org/httpcomponents-client +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents Name: httpcore Version: 4.4.16 +Project URL (from POM): http://hc.apache.org/httpcomponents-core-ga +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.client5 Name: httpclient5 Version: 5.3.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.core5 Name: httpcore5 Version: 5.2.4 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.core5 Name: httpcore5-h2 Version: 5.2.4 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-admin Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-client Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-common Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-core Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-crypto Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-identity Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-server Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-simplekdc Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-util Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-asn1 Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-config Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-pkix Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-util Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-xdr Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: token-provider Version: 1.0.1 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.orc Name: orc-core Version: 1.9.4 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.orc Name: orc-shims Version: 1.9.4 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-avro Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-column Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-common Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-encoding Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-format-structures Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org/ +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-hadoop Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-jackson Version: 1.13.1 +Project URL (from POM): https://parquet.apache.org +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.thrift Name: libfb303 Version: 0.9.3 +Project URL (from POM): http://thrift.apache.org +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.thrift Name: libthrift Version: 0.9.3 +Project URL (from POM): http://thrift.apache.org +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.yetus Name: audience-annotations Version: 0.13.0 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.zookeeper Name: zookeeper Version: 3.6.3 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.zookeeper Name: zookeeper-jute Version: 3.6.3 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.checkerframework Name: checker-qual Version: 3.44.0 +Manifest License: MIT (Not packaged) +Project URL (from POM): https://checkerframework.org/ +License (from POM): The MIT License - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: org.codehaus.jettison Name: jettison Version: 1.5.4 +Project URL (from POM): https://github.com/jettison-json/jettison +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: org.codehaus.mojo Name: animal-sniffer-annotations Version: 1.23 +License (from POM): MIT license - https://spdx.org/licenses/MIT.txt +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.codehaus.woodstox Name: stax2-api Version: 4.2.2 +Project URL (from manifest): http://github.com/FasterXML/stax2-api +Project URL (from POM): http://github.com/FasterXML/stax2-api +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): The BSD 2-Clause License - http://www.opensource.org/licenses/bsd-license.php + +-------------------------------------------------------------------------------- + +Group: org.conscrypt Name: conscrypt-openjdk-uber Version: 2.5.2 +Project URL (from POM): https://conscrypt.org/ +License (from POM): Apache 2 - https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-api-jdo Version: 4.2.4 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-core Version: 4.1.17 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-rdbms Version: 4.1.19 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: javax.jdo Version: 3.2.0-m3 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-client Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-http Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-io Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-security Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-servlet Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-util Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-util-ajax Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-webapp Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-xml Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty.websocket Name: websocket-api Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty.websocket Name: websocket-client Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty.websocket Name: websocket-common Version: 9.4.51.v20230217 +Project URL (from manifest): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - https://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.jetbrains Name: annotations Version: 17.0.0 +Project URL (from POM): https://github.com/JetBrains/java-annotations +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.jetbrains.kotlin Name: kotlin-stdlib Version: 1.4.10 +Project URL (from POM): https://kotlinlang.org/ +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.jetbrains.kotlin Name: kotlin-stdlib-common Version: 1.4.10 +Project URL (from POM): https://kotlinlang.org/ +License (from POM): The Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.jline Name: jline Version: 3.9.0 +License (from POM): The BSD License - http://www.opensource.org/licenses/bsd-license.php + +-------------------------------------------------------------------------------- + +Group: org.ow2.asm Name: asm Version: 9.3 +Project URL (from manifest): http://asm.ow2.org +Manifest License: BSD-3-Clause;link=https://asm.ow2.io/LICENSE.txt (Not packaged) +Project URL (from POM): http://asm.ow2.io/ +License (from POM): BSD-3-Clause - https://asm.ow2.io/license.html +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.reactivestreams Name: reactive-streams Version: 1.0.4 +Project URL (from manifest): http://reactive-streams.org +Project URL (from POM): http://www.reactive-streams.org/ +License (from POM): MIT-0 - https://spdx.org/licenses/MIT-0.html + +-------------------------------------------------------------------------------- + +Group: org.roaringbitmap Name: RoaringBitmap Version: 1.2.1 +Project URL (from POM): https://github.com/RoaringBitmap/RoaringBitmap +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.slf4j Name: slf4j-api Version: 2.0.13 +Project URL (from manifest): http://www.slf4j.org +Project URL (from POM): http://www.slf4j.org +License (from POM): MIT License - http://www.opensource.org/licenses/mit-license.php + +-------------------------------------------------------------------------------- + +Group: org.slf4j Name: slf4j-reload4j Version: 1.7.36 +Project URL (from POM): http://reload4j.qos.ch +License (from POM): MIT License - http://www.opensource.org/licenses/mit-license.php + +-------------------------------------------------------------------------------- + +Group: org.threeten Name: threeten-extra Version: 1.7.1 +Project URL (from manifest): https://www.threeten.org +Project URL (from POM): https://www.threeten.org/threeten-extra +License (from POM): BSD 3-clause - https://raw.githubusercontent.com/ThreeTen/threeten-extra/master/LICENSE.txt + +-------------------------------------------------------------------------------- + +Group: org.threeten Name: threetenbp Version: 1.6.9 +Project URL (from manifest): https://www.threeten.org +Project URL (from POM): https://www.threeten.org/threetenbp +License (from POM): BSD-3-Clause - https://raw.githubusercontent.com/ThreeTen/threetenbp/main/LICENSE.txt + +-------------------------------------------------------------------------------- + +Group: org.xerial.snappy Name: snappy-java Version: 1.1.10.5 +Project URL (from manifest): http://www.xerial.org/ +Project URL (from POM): https://github.com/xerial/snappy-java +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: annotations Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: apache-client Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: arns Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: auth Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: checksums Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: checksums-spi Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: crt-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: dynamodb Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: endpoints-spi Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: glue Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth-aws Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth-spi Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-client-spi Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: iam Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: identity-spi Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: json-utils Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: kms Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: lakeformation Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: metrics-spi Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: netty-nio-client Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: profiles Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: protocol-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: regions Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: retries Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: retries-spi Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: s3 Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sdk-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sso Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sts Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.26.29 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: utils Version: 2.26.29 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.eventstream Name: eventstream Version: 1.0.1 +Project URL (from POM): https://github.com/awslabs/aws-eventstream-java +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- diff --git a/kafka-connect/kafka-connect-runtime/NOTICE b/kafka-connect/kafka-connect-runtime/NOTICE new file mode 100644 index 000000000000..aad25386f10b --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/NOTICE @@ -0,0 +1,1723 @@ + +Apache Iceberg +Copyright 2017-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This project includes code from Kite, developed at Cloudera, Inc. with +the following copyright notice: + +| Copyright 2013 Cloudera Inc. +| +| Licensed 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. + +-------------------------------------------------------------------------------- + +This binary artifact contains code from the following projects: + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-math3 Version: 3.1.1 + +Notice: Apache Commons Math +Copyright 2001-2012 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + +=============================================================================== + +The BracketFinder (package org.apache.commons.math3.optimization.univariate) +and PowellOptimizer (package org.apache.commons.math3.optimization.general) +classes are based on the Python code in module "optimize.py" (version 0.5) +developed by Travis E. Oliphant for the SciPy library (http://www.scipy.org/) +Copyright © 2003-2009 SciPy Developers. +=============================================================================== + +The LinearConstraint, LinearObjectiveFunction, LinearOptimizer, +RelationShip, SimplexSolver and SimplexTableau classes in package +org.apache.commons.math3.optimization.linear include software developed by +Benjamin McCann (http://www.benmccann.com) and distributed with +the following copyright: Copyright 2009 Google Inc. +=============================================================================== + +This product includes software developed by the +University of Chicago, as Operator of Argonne National +Laboratory. +The LevenbergMarquardtOptimizer class in package +org.apache.commons.math3.optimization.general includes software +translated from the lmder, lmpar and qrsolv Fortran routines +from the Minpack package +Minpack Copyright Notice (1999) University of Chicago. All rights reserved +=============================================================================== + +The GraggBulirschStoerIntegrator class in package +org.apache.commons.math3.ode.nonstiff includes software translated +from the odex Fortran routine developed by E. Hairer and G. Wanner. +Original source copyright: +Copyright (c) 2004, Ernst Hairer +=============================================================================== + +The EigenDecompositionImpl class in package +org.apache.commons.math3.linear includes software translated +from some LAPACK Fortran routines. Original source copyright: +Copyright (c) 1992-2008 The University of Tennessee. All rights reserved. +=============================================================================== + +The MersenneTwister class in package org.apache.commons.math3.random +includes software translated from the 2002-01-26 version of +the Mersenne-Twister generator written in C by Makoto Matsumoto and Takuji +Nishimura. Original source copyright: +Copyright (C) 1997 - 2002, Makoto Matsumoto and Takuji Nishimura, +All rights reserved +=============================================================================== + +The LocalizedFormatsTest class in the unit tests is an adapted version of +the OrekitMessagesTest class from the orekit library distributed under the +terms of the Apache 2 licence. Original source copyright: +Copyright 2010 CS Systèmes d'Information +=============================================================================== + +The HermiteInterpolator class and its corresponding test have been imported from +the orekit library distributed under the terms of the Apache 2 licence. Original +source copyright: +Copyright 2010-2012 CS Systèmes d'Information +=============================================================================== + +The creation of the package "o.a.c.m.analysis.integration.gauss" was inspired +by an original code donated by Sébastien Brisard. +=============================================================================== + + +The complete text of licenses and disclaimers associated with the the original +sources enumerated above at the time of code translation are in the LICENSE.txt +file. + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.core5 Name: httpcore5-h2 Version: 5.2.4 + +Notice: Apache HttpComponents Core HTTP/2 +Copyright 2005-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-admin Version: 1.0.1 + +Notice: Kerby-kerb Admin +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-crypto Version: 1.0.1 + +Notice: Kerby-kerb Crypto +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.orc Name: orc-core Version: 1.9.4 + +Notice: ORC Core +Copyright 2013-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-shims Version: 2.3.9 + +Notice: Hive Shims +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-pkix Version: 1.0.1 + +Notice: Kerby PKIX Project +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-scheduler Version: 2.3.9 + +Notice: Hive Shims Scheduler +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.ant Name: ant Version: 1.9.1 +Group: org.apache.ant Name: ant-launcher Version: 1.9.1 + +Notice: Apache Ant + Copyright 1999-2013 The Apache Software Foundation + + The task is based on code Copyright (c) 2002, Landmark + Graphics Corp that has been kindly donated to the Apache Software + Foundation. + + +-------------------------------------------------------------------------------- + +Group: commons-lang Name: commons-lang Version: 2.6 + +Notice: Apache Commons Lang +Copyright 2001-2011 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-configuration2 Version: 2.8.0 + +Notice: Apache Commons Configuration +Copyright 2001-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-config Version: 1.0.1 + +Notice: Kerby Config +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-avro Version: 1.13.1 + +Notice: Apache Parquet MR (Incubating) +Copyright 2014-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This product includes code from Apache Avro, which includes the following in +its NOTICE file: + + Apache Avro + Copyright 2010-2015 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: io.grpc Name: grpc-netty-shaded Version: 1.62.2 + +Notice: The Netty Project + ================= + +Please visit the Netty web site for more information: + + * http://netty.io/ + +Copyright 2016 The Netty Project + +The Netty Project 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. + +------------------------------------------------------------------------------- +This product contains a forked and modified version of Tomcat Native + + * LICENSE: + * license/LICENSE.tomcat-native.txt (Apache License 2.0) + * HOMEPAGE: + * http://tomcat.apache.org/native-doc/ + * https://svn.apache.org/repos/asf/tomcat/native/ + +This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build. + + * LICENSE: + * license/LICENSE.mvn-wrapper.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/takari/maven-wrapper + +This product contains small piece of code to support AIX, taken from netbsd. + + * LICENSE: + * license/LICENSE.aix-netbsd.txt (OpenSSL License) + * HOMEPAGE: + * https://ftp.netbsd.org/pub/NetBSD/NetBSD-current/src/crypto/external/bsd/openssl/dist + + +This product contains code from boringssl. + + * LICENSE (Combination ISC and OpenSSL license) + * license/LICENSE.boringssl.txt (Combination ISC and OpenSSL license) + * HOMEPAGE: + * https://boringssl.googlesource.com/boringssl/ + + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-recipes Version: 5.2.0 + +Notice: Curator Recipes +Copyright 2011-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-storage-api Version: 2.4.0 + +Notice: Hive Storage API +Copyright 2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-annotations Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-auth Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-client Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-common Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-hdfs-client Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-mapreduce-client-common Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-mapreduce-client-core Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-mapreduce-client-jobclient Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-yarn-api Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-yarn-client Version: 3.3.6 +Group: org.apache.hadoop Name: hadoop-yarn-common Version: 3.3.6 + +Notice: Apache Hadoop +Copyright 2006 and onwards The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Export Control Notice +--------------------- + +This distribution includes cryptographic software. The country in +which you currently reside may have restrictions on the import, +possession, use, and/or re-export to another country, of +encryption software. BEFORE using any encryption software, please +check your country's laws, regulations and policies concerning the +import, possession, or use, and re-export of encryption software, to +see if this is permitted. See for more +information. + +The U.S. Government Department of Commerce, Bureau of Industry and +Security (BIS), has classified this software as Export Commodity +Control Number (ECCN) 5D002.C.1, which includes information security +software using or performing cryptographic functions with asymmetric +algorithms. The form and manner of this Apache Software Foundation +distribution makes it eligible for export under the License Exception +ENC Technology Software Unrestricted (TSU) exception (see the BIS +Export Administration Regulations, Section 740.13) for both object +code and source code. + +The following provides more details on the included cryptographic software: + +This software uses the SSL libraries from the Jetty project written +by mortbay.org. +Hadoop Yarn Server Web Proxy uses the BouncyCastle Java +cryptography APIs written by the Legion of the Bouncy Castle Inc. + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.dataformat Name: jackson-dataformat-xml Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Copyright + +Copyright 2007-, Tatu Saloranta (tatu.saloranta@iki.fi) + +## Licensing + +Jackson components are licensed under Apache (Software) License, version 2.0, +as per accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-common Version: 2.3.9 + +Notice: Hive Common +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-common Version: 1.0.1 + +Notice: Kerby-kerb Common +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-xdr Version: 1.0.1 + +Notice: Kerby XDR Project +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.parquet Name: parquet-jackson Version: 1.13.1 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Licensing + +Jackson 2.x core and extension components are licensed under Apache License 2.0 +To find the details that apply to this artifact see the accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS(-2.x) file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: jakarta.activation Name: jakarta.activation-api Version: 1.2.2 + +Notice: # Notices for Jakarta Activation + +This content is produced and maintained by Jakarta Activation project. + +* Project home: https://projects.eclipse.org/projects/ee4j.jaf + +## Copyright + +All content is the property of the respective authors or their employers. For +more information regarding authorship of content, please consult the listed +source code repository logs. + +## Declared Project Licenses + +This program and the accompanying materials are made available under the terms +of the Eclipse Distribution License v. 1.0, +which is available at http://www.eclipse.org/org/documents/edl-v10.php. + +SPDX-License-Identifier: BSD-3-Clause + +## Source Code + +The project maintains the following source code repositories: + +* https://github.com/eclipse-ee4j/jaf + +## Third-party Content + +This project leverages the following third party content. + +JUnit (4.12) + +* License: Eclipse Public License + + +-------------------------------------------------------------------------------- + +Group: jakarta.xml.bind Name: jakarta.xml.bind-api Version: 2.3.3 + +Notice: [//]: # " Copyright (c) 2018, 2019 Oracle and/or its affiliates. All rights reserved. " +[//]: # " " +[//]: # " This program and the accompanying materials are made available under the " +[//]: # " terms of the Eclipse Distribution License v. 1.0, which is available at " +[//]: # " http://www.eclipse.org/org/documents/edl-v10.php. " +[//]: # " " +[//]: # " SPDX-License-Identifier: BSD-3-Clause " + +# Notices for Jakarta XML Binding + +This content is produced and maintained by the Jakarta XML Binding +project. + +* Project home: https://projects.eclipse.org/projects/ee4j.jaxb + +## Trademarks + +Jakarta XML Binding is a trademark of the Eclipse Foundation. + +## Copyright + +All content is the property of the respective authors or their employers. For +more information regarding authorship of content, please consult the listed +source code repository logs. + +## Declared Project Licenses + +This program and the accompanying materials are made available under the terms +of the Eclipse Distribution License v. 1.0 which is available at +http://www.eclipse.org/org/documents/edl-v10.php. + +SPDX-License-Identifier: BSD-3-Clause + +## Source Code + +The project maintains the following source code repositories: + +* https://github.com/eclipse-ee4j/jaxb-api +* https://github.com/eclipse-ee4j/jaxb-tck + +## Third-party Content + +This project leverages the following third party content. + +Apache River (3.0.0) + +* License: Apache-2.0 AND BSD-3-Clause + +ASM 7 (n/a) + +* License: BSD-3-Clause +* Project: https://asm.ow2.io/ +* Source: + https://repository.ow2.org/nexus/#nexus-search;gav~org.ow2.asm~asm-commons~~~~kw,versionexpand + +JTHarness (5.0) + +* License: (GPL-2.0 OR GPL-2.0 WITH Classpath-exception-2.0) +* Project: https://wiki.openjdk.java.net/display/CodeTools/JT+Harness +* Source: http://hg.openjdk.java.net/code-tools/jtharness/ + +normalize.css (3.0.2) + +* License: MIT + +SigTest (n/a) + +* License: GPL-2.0 OR GPL-2.0 WITH Classpath-exception-2.0 + +## Cryptography + +Content may contain encryption software. The country in which you are currently +may have restrictions on the import, possession, and use, and/or re-export to +another country, of encryption software. BEFORE using any encryption software, +please check the country's laws, regulations and policies concerning the import, +possession, or use, and re-export of encryption software, to see if this is +permitted. + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.module Name: jackson-module-jaxb-annotations Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-compress Version: 1.26.2 + +Notice: Apache Commons Compress +Copyright 2002-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-client Version: 1.0.1 + +Notice: Kerby-kerb Client +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-io Name: commons-io Version: 2.16.1 + +Notice: Apache Commons IO +Copyright 2002-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-core Version: 4.1.17 + +Notice: ========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the DataNucleus distribution. == +========================================================================= + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Erik Bengtson +Andy Jefferson + + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Joerg von Frantzius +Thomas Marti +Barry Haddow +Marco Schulze +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Marcus Mennemeier +Xuan Baldauf +Eric Sultan + + +=================================================================== +This product also includes software developed by the TJDO project +(http://tjdo.sourceforge.net/). +=================================================================== + + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: annotations Version: 2.26.29 +Group: software.amazon.awssdk Name: apache-client Version: 2.26.29 +Group: software.amazon.awssdk Name: arns Version: 2.26.29 +Group: software.amazon.awssdk Name: auth Version: 2.26.29 +Group: software.amazon.awssdk Name: aws-core Version: 2.26.29 +Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.26.29 +Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.26.29 +Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.26.29 +Group: software.amazon.awssdk Name: checksums Version: 2.26.29 +Group: software.amazon.awssdk Name: checksums-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: crt-core Version: 2.26.29 +Group: software.amazon.awssdk Name: dynamodb Version: 2.26.29 +Group: software.amazon.awssdk Name: endpoints-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: glue Version: 2.26.29 +Group: software.amazon.awssdk Name: http-auth Version: 2.26.29 +Group: software.amazon.awssdk Name: http-auth-aws Version: 2.26.29 +Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.26.29 +Group: software.amazon.awssdk Name: http-auth-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: http-client-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: iam Version: 2.26.29 +Group: software.amazon.awssdk Name: identity-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: json-utils Version: 2.26.29 +Group: software.amazon.awssdk Name: kms Version: 2.26.29 +Group: software.amazon.awssdk Name: lakeformation Version: 2.26.29 +Group: software.amazon.awssdk Name: metrics-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: netty-nio-client Version: 2.26.29 +Group: software.amazon.awssdk Name: profiles Version: 2.26.29 +Group: software.amazon.awssdk Name: protocol-core Version: 2.26.29 +Group: software.amazon.awssdk Name: regions Version: 2.26.29 +Group: software.amazon.awssdk Name: retries Version: 2.26.29 +Group: software.amazon.awssdk Name: retries-spi Version: 2.26.29 +Group: software.amazon.awssdk Name: s3 Version: 2.26.29 +Group: software.amazon.awssdk Name: sdk-core Version: 2.26.29 +Group: software.amazon.awssdk Name: sso Version: 2.26.29 +Group: software.amazon.awssdk Name: sts Version: 2.26.29 +Group: software.amazon.awssdk Name: utils Version: 2.26.29 + +Notice: AWS SDK for Java 2.0 +Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. + +This product includes software developed by +Amazon Technologies, Inc (http://www.amazon.com/). + +********************** +THIRD PARTY COMPONENTS +********************** +This software includes third party software subject to the following copyrights: +- XML parsing and utility functions from JetS3t - Copyright 2006-2009 James Murty. +- PKCS#1 PEM encoded private key parsing and utility functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc. +- Apache Commons Lang - https://github.com/apache/commons-lang +- Netty Reactive Streams - https://github.com/playframework/netty-reactive-streams +- Jackson-core - https://github.com/FasterXML/jackson-core +- Jackson-dataformat-cbor - https://github.com/FasterXML/jackson-dataformats-binary + +The licenses for these third party components are included in LICENSE.txt + +- For Apache Commons Lang see also this required NOTICE: + Apache Commons Lang + Copyright 2001-2020 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.client5 Name: httpclient5 Version: 5.3.1 + +Notice: Apache HttpClient +Copyright 1999-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-common Version: 2.3.9 + +Notice: Hive Shims Common +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-logging Name: commons-logging Version: 1.2 + +Notice: Apache Commons Logging +Copyright 2003-2014 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-dbcp Name: commons-dbcp Version: 1.4 + +Notice: Apache Commons DBCP +Copyright 2001-2010 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-net Name: commons-net Version: 3.9.0 + +Notice: Apache Commons Net +Copyright 2001-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: com.squareup.okhttp3 Name: okhttp Version: 4.9.3 + +Notice: Note that publicsuffixes.gz is compiled from The Public Suffix List: +https://publicsuffix.org/list/public_suffix_list.dat + +It is subject to the terms of the Mozilla Public License, v. 2.0: +https://mozilla.org/MPL/2.0/ + + +-------------------------------------------------------------------------------- + +Group: commons-collections Name: commons-collections Version: 3.2.2 + +Notice: Apache Commons Collections +Copyright 2001-2015 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: jakarta.activation Name: jakarta.activation-api Version: 1.2.1 + +Notice: # Notices for Eclipse Project for JAF + +This content is produced and maintained by the Eclipse Project for JAF project. + +* Project home: https://projects.eclipse.org/projects/ee4j.jaf + +## Copyright + +All content is the property of the respective authors or their employers. For +more information regarding authorship of content, please consult the listed +source code repository logs. + +## Declared Project Licenses + +This program and the accompanying materials are made available under the terms +of the Eclipse Distribution License v. 1.0, +which is available at http://www.eclipse.org/org/documents/edl-v10.php. + +SPDX-License-Identifier: BSD-3-Clause + +## Source Code + +The project maintains the following source code repositories: + +* https://github.com/eclipse-ee4j/jaf + +## Third-party Content + +This project leverages the following third party content. + +JUnit (4.12) + +* License: Eclipse Public License + + +-------------------------------------------------------------------------------- + +Group: org.apache.derby Name: derby Version: 10.10.2.0 + +Notice: ========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, +== Version 2.0, in this case for the Apache Derby distribution. +== +== DO NOT EDIT THIS FILE DIRECTLY. IT IS GENERATED +== BY THE buildnotice TARGET IN THE TOP LEVEL build.xml FILE. +== +========================================================================= + +Apache Derby +Copyright 2004-2014 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +========================================================================= + +Portions of Derby were originally developed by +International Business Machines Corporation and are +licensed to the Apache Software Foundation under the +"Software Grant and Corporate Contribution License Agreement", +informally known as the "Derby CLA". +The following copyright notice(s) were affixed to portions of the code +with which this file is now or was at one time distributed +and are placed here unaltered. + +(C) Copyright 1997,2004 International Business Machines Corporation. All rights reserved. + +(C) Copyright IBM Corp. 2003. + + +========================================================================= + + +The portion of the functionTests under 'nist' was originally +developed by the National Institute of Standards and Technology (NIST), +an agency of the United States Department of Commerce, and adapted by +International Business Machines Corporation in accordance with the NIST +Software Acknowledgment and Redistribution document at +http://www.itl.nist.gov/div897/ctg/sql_form.htm + + + +========================================================================= + + +The JDBC apis for small devices and JDBC3 (under java/stubs/jsr169 and +java/stubs/jdbc3) were produced by trimming sources supplied by the +Apache Harmony project. In addition, the Harmony SerialBlob and +SerialClob implementations are used. The following notice covers the Harmony sources: + +Portions of Harmony were originally developed by +Intel Corporation and are licensed to the Apache Software +Foundation under the "Software Grant and Corporate Contribution +License Agreement", informally known as the "Intel Harmony CLA". + + +========================================================================= + + +The Derby build relies on source files supplied by the Apache Felix +project. The following notice covers the Felix files: + + Apache Felix Main + Copyright 2008 The Apache Software Foundation + + + I. Included Software + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + Licensed under the Apache License 2.0. + + This product includes software developed at + The OSGi Alliance (http://www.osgi.org/). + Copyright (c) OSGi Alliance (2000, 2007). + Licensed under the Apache License 2.0. + + This product includes software from http://kxml.sourceforge.net. + Copyright (c) 2002,2003, Stefan Haustein, Oberhausen, Rhld., Germany. + Licensed under BSD License. + + II. Used Software + + This product uses software developed at + The OSGi Alliance (http://www.osgi.org/). + Copyright (c) OSGi Alliance (2000, 2007). + Licensed under the Apache License 2.0. + + + III. License Summary + - Apache License 2.0 + - BSD License + + +========================================================================= + + +The Derby build relies on jar files supplied by the Apache Xalan +project. The following notice covers the Xalan jar files: + + ========================================================================= + == NOTICE file corresponding to section 4(d) of the Apache License, == + == Version 2.0, in this case for the Apache Xalan Java distribution. == + ========================================================================= + + Apache Xalan (Xalan XSLT processor) + Copyright 1999-2006 The Apache Software Foundation + + Apache Xalan (Xalan serializer) + Copyright 1999-2006 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + ========================================================================= + Portions of this software was originally based on the following: + - software copyright (c) 1999-2002, Lotus Development Corporation., + http://www.lotus.com. + - software copyright (c) 2001-2002, Sun Microsystems., + http://www.sun.com. + - software copyright (c) 2003, IBM Corporation., + http://www.ibm.com. + + ========================================================================= + The binary distribution package (ie. jars, samples and documentation) of + this product includes software developed by the following: + + - The Apache Software Foundation + - Xerces Java - see LICENSE.txt + - JAXP 1.3 APIs - see LICENSE.txt + - Bytecode Engineering Library - see LICENSE.txt + - Regular Expression - see LICENSE.txt + + - Scott Hudson, Frank Flannery, C. Scott Ananian + - CUP Parser Generator runtime (javacup\runtime) - see LICENSE.txt + + ========================================================================= + The source distribution package (ie. all source and tools required to build + Xalan Java) of this product includes software developed by the following: + + - The Apache Software Foundation + - Xerces Java - see LICENSE.txt + - JAXP 1.3 APIs - see LICENSE.txt + - Bytecode Engineering Library - see LICENSE.txt + - Regular Expression - see LICENSE.txt + - Ant - see LICENSE.txt + - Stylebook doc tool - see LICENSE.txt + + - Elliot Joel Berk and C. Scott Ananian + - Lexical Analyzer Generator (JLex) - see LICENSE.txt + + ========================================================================= + Apache Xerces Java + Copyright 1999-2006 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Portions of Apache Xerces Java in xercesImpl.jar and xml-apis.jar + were originally based on the following: + - software copyright (c) 1999, IBM Corporation., http://www.ibm.com. + - software copyright (c) 1999, Sun Microsystems., http://www.sun.com. + - voluntary contributions made by Paul Eng on behalf of the + Apache Software Foundation that were originally developed at iClick, Inc., + software copyright (c) 1999. + + ========================================================================= + Apache xml-commons xml-apis (redistribution of xml-apis.jar) + + Apache XML Commons + Copyright 2001-2003,2006 The Apache Software Foundation. + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Portions of this software were originally based on the following: + - software copyright (c) 1999, IBM Corporation., http://www.ibm.com. + - software copyright (c) 1999, Sun Microsystems., http://www.sun.com. + - software copyright (c) 2000 World Wide Web Consortium, http://www.w3.org + + +-------------------------------------------------------------------------------- + +Group: com.google.inject Name: guice Version: 4.0 + +Notice: Google Guice - Core Library +Copyright 2006-2015 Google, Inc. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: ch.qos.reload4j Name: reload4j Version: 1.2.22 +Group: log4j Name: log4j Version: 1.2.17 + +Notice: Apache log4j +Copyright 2007 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-codec Name: commons-codec Version: 1.17.1 + +Notice: Apache Commons Codec +Copyright 2002-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: joda-time Name: joda-time Version: 2.8.1 + +Notice: ============================================================================= += NOTICE file corresponding to section 4d of the Apache License Version 2.0 = +============================================================================= +This product includes software developed by +Joda.org (http://www.joda.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-text Version: 1.10.0 + +Notice: Apache Commons Text +Copyright 2014-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-identity Version: 1.0.1 + +Notice: Kerby-kerb Identity +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-client Version: 5.2.0 + +Notice: Curator Client +Copyright 2011-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.thrift Name: libthrift Version: 0.9.3 + +Notice: Apache Thrift +Copyright 2006-2010 The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-core Version: 2.17.2 +Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.26.29 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Copyright + +Copyright 2007-, Tatu Saloranta (tatu.saloranta@iki.fi) + +## Licensing + +Jackson 2.x core and extension components are licensed under Apache License 2.0 +To find the details that apply to this artifact see the accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS(-2.x) file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + +## FastDoubleParser + +jackson-core bundles a shaded copy of FastDoubleParser . +That code is available under an MIT license +under the following copyright. + +Copyright © 2023 Werner Randelshofer, Switzerland. MIT License. + +See FastDoubleParser-NOTICE for details of other source code included in FastDoubleParser +and the licenses and copyrights that apply to that code. + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.datatype Name: jackson-datatype-jsr310 Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Licensing + +Jackson components are licensed under Apache (Software) License, version 2.0, +as per accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.curator Name: curator-framework Version: 5.2.0 + +Notice: Curator Framework +Copyright 2011-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents Name: httpclient Version: 4.5.13 + +Notice: Apache HttpClient +Copyright 1999-2020 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-util Version: 1.0.1 + +Notice: Kerby Util +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-serde Version: 2.3.9 + +Notice: Hive Serde +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.core5 Name: httpcore5 Version: 5.2.4 + +Notice: Apache HttpComponents Core HTTP/1.1 +Copyright 2005-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-core Version: 1.0.1 + +Notice: Kerby-kerb core +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-api-jdo Version: 4.2.4 + +Notice: ========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the DataNucleus distribution. == +========================================================================= + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Erik Bengtson +Andy Jefferson + + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-util Version: 1.0.1 + +Notice: Kerby-kerb Util +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.datanucleus Name: datanucleus-rdbms Version: 4.1.19 + +Notice: ========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the DataNucleus distribution. == +========================================================================= + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Andy Jefferson +Erik Bengtson +Joerg von Frantzius +Marco Schulze + + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Barry Haddow +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Anton Troshin (Timesten) + + +=================================================================== +This product also includes software developed by the TJDO project +(http://tjdo.sourceforge.net/). +=================================================================== + +=================================================================== +This product also includes software developed by the Apache Commons project +(http://commons.apache.org/). +=================================================================== + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive Name: hive-metastore Version: 2.3.9 + +Notice: Hive Metastore +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-cli Name: commons-cli Version: 1.2 + +Notice: Apache Commons CLI +Copyright 2001-2009 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-pool Name: commons-pool Version: 1.6 + +Notice: Apache Commons Pool +Copyright 2001-2012 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: javax.jdo Name: jdo-api Version: 3.0.1 + +Notice: Apache Java Data Objects (JDO) +Copyright 2005-2006 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.orc Name: orc-shims Version: 1.9.4 + +Notice: ORC Shims +Copyright 2013-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop.thirdparty Name: hadoop-shaded-guava Version: 1.2.0 +Group: org.apache.hadoop.thirdparty Name: hadoop-shaded-protobuf_3_7 Version: 1.1.1 + +Notice: Apache Hadoop Third-party Libs +Copyright 2020 and onwards The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents Name: httpcore Version: 4.4.16 + +Notice: Apache HttpCore +Copyright 2005-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: commons-beanutils Name: commons-beanutils Version: 1.9.4 + +Notice: Apache Commons BeanUtils +Copyright 2000-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-annotations Version: 2.17.2 +Group: com.fasterxml.jackson.core Name: jackson-databind Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Copyright + +Copyright 2007-, Tatu Saloranta (tatu.saloranta@iki.fi) + +## Licensing + +Jackson 2.x core and extension components are licensed under Apache License 2.0 +To find the details that apply to this artifact see the accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS(-2.x) file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-simplekdc Version: 1.0.1 + +Notice: Kerb Simple Kdc +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerb-server Version: 1.0.1 + +Notice: Kerby-kerb Server +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.avro Name: avro Version: 1.12.0 + +Notice: Apache Avro +Copyright 2009-2024 The Apache Software Foundation + + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.hive.shims Name: hive-shims-0.23 Version: 2.3.9 + +Notice: Hive Shims 0.23 +Copyright 2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: kerby-asn1 Version: 1.0.1 + +Notice: Kerby ASN1 Project +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.kerby Name: token-provider Version: 1.0.1 + +Notice: Token provider +Copyright 2014-2017 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.jaxrs Name: jackson-jaxrs-json-provider Version: 2.17.2 + +Notice: # Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may be licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + + +-------------------------------------------------------------------------------- + +Group: org.apache.yetus Name: audience-annotations Version: 0.13.0 + +Notice: Apache Yetus - Audience Annotations +Copyright 2015-2020 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-lang3 Version: 3.14.0 + +Notice: Apache Commons Lang +Copyright 2001-2023 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-client Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-http Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-io Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-security Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-servlet Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-util Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-util-ajax Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-webapp Version: 9.4.51.v20230217 +Group: org.eclipse.jetty Name: jetty-xml Version: 9.4.51.v20230217 +Group: org.eclipse.jetty.websocket Name: websocket-api Version: 9.4.51.v20230217 +Group: org.eclipse.jetty.websocket Name: websocket-client Version: 9.4.51.v20230217 +Group: org.eclipse.jetty.websocket Name: websocket-common Version: 9.4.51.v20230217 + +Notice: ============================================================== + Jetty Web Container + Copyright 1995-2018 Mort Bay Consulting Pty Ltd. +============================================================== + +The Jetty Web Container is Copyright Mort Bay Consulting Pty Ltd +unless otherwise noted. + +Jetty is dual licensed under both + + * The Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0.html + + and + + * The Eclipse Public 1.0 License + http://www.eclipse.org/legal/epl-v10.html + +Jetty may be distributed under either license. + +------ +Eclipse + +The following artifacts are EPL. + * org.eclipse.jetty.orbit:org.eclipse.jdt.core + +The following artifacts are EPL and ASL2. + * org.eclipse.jetty.orbit:javax.security.auth.message + + +The following artifacts are EPL and CDDL 1.0. + * org.eclipse.jetty.orbit:javax.mail.glassfish + + +------ +Oracle + +The following artifacts are CDDL + GPLv2 with classpath exception. +https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + * javax.servlet:javax.servlet-api + * javax.annotation:javax.annotation-api + * javax.transaction:javax.transaction-api + * javax.websocket:javax.websocket-api + +------ +Oracle OpenJDK + +If ALPN is used to negotiate HTTP/2 connections, then the following +artifacts may be included in the distribution or downloaded when ALPN +module is selected. + + * java.sun.security.ssl + +These artifacts replace/modify OpenJDK classes. The modififications +are hosted at github and both modified and original are under GPL v2 with +classpath exceptions. +http://openjdk.java.net/legal/gplv2+ce.html + + +------ +OW2 + +The following artifacts are licensed by the OW2 Foundation according to the +terms of http://asm.ow2.org/license.html + +org.ow2.asm:asm-commons +org.ow2.asm:asm + + +------ +Apache + +The following artifacts are ASL2 licensed. + +org.apache.taglibs:taglibs-standard-spec +org.apache.taglibs:taglibs-standard-impl + + +------ +MortBay + +The following artifacts are ASL2 licensed. Based on selected classes from +following Apache Tomcat jars, all ASL2 licensed. + +org.mortbay.jasper:apache-jsp + org.apache.tomcat:tomcat-jasper + org.apache.tomcat:tomcat-juli + org.apache.tomcat:tomcat-jsp-api + org.apache.tomcat:tomcat-el-api + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-api + org.apache.tomcat:tomcat-util-scan + org.apache.tomcat:tomcat-util + +org.mortbay.jasper:apache-el + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-el-api + + +------ +Mortbay + +The following artifacts are CDDL + GPLv2 with classpath exception. + +https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + +org.eclipse.jetty.toolchain:jetty-schemas + +------ +Assorted + +The UnixCrypt.java code implements the one way cryptography used by +Unix systems for simple password protection. Copyright 1996 Aki Yoshida, +modified April 2001 by Iris Van den Broeke, Daniel Deville. +Permission to use, copy, modify and distribute UnixCrypt +for non-commercial or commercial purposes and without fee is +granted provided that the copyright notice appears in all copies. + + +-------------------------------------------------------------------------------- From fdc2c223efa3367f8a10d9c49985326352b687b4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Thu, 24 Oct 2024 20:55:18 +0200 Subject: [PATCH 040/313] Deprecate iceberg-pig (#11379) --- .../apache/iceberg/pig/IcebergPigInputFormat.java | 5 +++++ .../java/org/apache/iceberg/pig/IcebergStorage.java | 8 ++++++++ .../org/apache/iceberg/pig/PigParquetReader.java | 13 ++++++++++++- .../java/org/apache/iceberg/pig/SchemaUtil.java | 4 ++++ 4 files changed, 29 insertions(+), 1 deletion(-) diff --git a/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java index 0d5a6dd9271f..932de72ac8c0 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java +++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java @@ -56,6 +56,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * @deprecated will be removed in 1.8.0 + */ +@Deprecated public class IcebergPigInputFormat extends InputFormat { private static final Logger LOG = LoggerFactory.getLogger(IcebergPigInputFormat.class); @@ -68,6 +72,7 @@ public class IcebergPigInputFormat extends InputFormat { private List splits; IcebergPigInputFormat(Table table, String signature) { + LOG.warn("Iceberg Pig is deprecated and will be removed in Iceberg 1.8.0"); this.table = table; this.signature = signature; } diff --git a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java index 88233c58a372..0ce23c39913a 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java +++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java @@ -64,6 +64,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * @deprecated will be removed in 1.8.0 + */ +@Deprecated public class IcebergStorage extends LoadFunc implements LoadMetadata, LoadPredicatePushdown, LoadPushDown { private static final Logger LOG = LoggerFactory.getLogger(IcebergStorage.class); @@ -77,6 +81,10 @@ public class IcebergStorage extends LoadFunc private IcebergRecordReader reader; + public IcebergStorage() { + LOG.warn("Iceberg Pig is deprecated and will be removed in Iceberg 1.8.0"); + } + @Override public void setLocation(String location, Job job) { LOG.info("[{}]: setLocation() -> {}", signature, location); diff --git a/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java b/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java index 35b061db4f9e..15ba9068caf5 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java +++ b/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java @@ -60,9 +60,20 @@ import org.apache.pig.data.DataByteArray; import org.apache.pig.data.Tuple; import org.apache.pig.data.TupleFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * @deprecated will be removed in 1.8.0 + */ +@Deprecated public class PigParquetReader { - private PigParquetReader() {} + + private static final Logger LOG = LoggerFactory.getLogger(PigParquetReader.class); + + private PigParquetReader() { + LOG.warn("Iceberg Pig is deprecated and will be removed in Iceberg 1.8.0"); + } @SuppressWarnings("unchecked") public static ParquetValueReader buildReader( diff --git a/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java b/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java index e02cfaf677b8..4602a5effa97 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java +++ b/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java @@ -29,6 +29,10 @@ import org.apache.pig.data.DataType; import org.apache.pig.impl.logicalLayer.FrontendException; +/** + * @deprecated will be removed in 1.8.0 + */ +@Deprecated public class SchemaUtil { private SchemaUtil() {} From 6b04a6d000019a6182eb6521d1e7e4124a0cd73b Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 25 Oct 2024 16:25:23 +0200 Subject: [PATCH 041/313] Core: Track data files by spec id instead of full PartitionSpec (#11323) --- .../iceberg/MergingSnapshotProducer.java | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index ab55f86ebf6f..50885dbb06c7 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -80,7 +80,7 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final ManifestFilterManager deleteFilterManager; // update data - private final Map newDataFilesBySpec = Maps.newHashMap(); + private final Map newDataFilesBySpec = Maps.newHashMap(); private Long newDataFilesDataSequenceNumber; private final Map newDeleteFilesBySpec = Maps.newHashMap(); private final List appendManifests = Lists.newArrayList(); @@ -138,18 +138,13 @@ protected boolean isCaseSensitive() { } protected PartitionSpec dataSpec() { - Set specs = dataSpecs(); + Set specIds = newDataFilesBySpec.keySet(); Preconditions.checkState( - specs.size() == 1, - "Cannot return a single partition spec: data files with different partition specs have been added"); - return specs.iterator().next(); - } - - protected Set dataSpecs() { - Set specs = newDataFilesBySpec.keySet(); + !specIds.isEmpty(), "Cannot determine partition specs: no data files have been added"); Preconditions.checkState( - !specs.isEmpty(), "Cannot determine partition specs: no data files have been added"); - return ImmutableSet.copyOf(specs); + specIds.size() == 1, + "Cannot return a single partition spec: data files with different partition specs have been added"); + return spec(Iterables.getOnlyElement(specIds)); } protected Expression rowFilter() { @@ -237,7 +232,7 @@ protected void add(DataFile file) { file.location()); DataFileSet dataFiles = - newDataFilesBySpec.computeIfAbsent(spec, ignored -> DataFileSet.create()); + newDataFilesBySpec.computeIfAbsent(spec.specId(), ignored -> DataFileSet.create()); if (dataFiles.add(file)) { addedFilesSummary.addedFile(spec, file); hasNewDataFiles = true; @@ -971,9 +966,9 @@ private List newDataFilesAsManifests() { if (cachedNewDataManifests.isEmpty()) { newDataFilesBySpec.forEach( - (dataSpec, dataFiles) -> { + (specId, dataFiles) -> { List newDataManifests = - writeDataManifests(dataFiles, newDataFilesDataSequenceNumber, dataSpec); + writeDataManifests(dataFiles, newDataFilesDataSequenceNumber, spec(specId)); cachedNewDataManifests.addAll(newDataManifests); }); this.hasNewDataFiles = false; From 32e9f40468756a60d2cc52e2b9e951209268e94b Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Sat, 26 Oct 2024 03:28:51 +0800 Subject: [PATCH 042/313] Spark 3.5: Don't change table distribution when only altering local order (#10774) --- .../IcebergSqlExtensionsAstBuilder.scala | 10 ++++--- .../SetWriteDistributionAndOrderingExec.scala | 10 ++++--- .../TestSetWriteDistributionAndOrdering.java | 29 +++++++++++++++++-- .../SetWriteDistributionAndOrdering.scala | 2 +- 4 files changed, 40 insertions(+), 11 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index 2e438de2b8cd..6b1cc41da04c 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -226,11 +226,13 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS } val distributionMode = if (distributionSpec != null) { - DistributionMode.HASH - } else if (orderingSpec.UNORDERED != null || orderingSpec.LOCALLY != null) { - DistributionMode.NONE + Some(DistributionMode.HASH) + } else if (orderingSpec.UNORDERED != null) { + Some(DistributionMode.NONE) + } else if (orderingSpec.LOCALLY() != null) { + None } else { - DistributionMode.RANGE + Some(DistributionMode.RANGE) } val ordering = if (orderingSpec != null && orderingSpec.order != null) { diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala index feecc0235076..c9004ddc5bda 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.connector.catalog.TableCatalog case class SetWriteDistributionAndOrderingExec( catalog: TableCatalog, ident: Identifier, - distributionMode: DistributionMode, + distributionMode: Option[DistributionMode], sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafV2CommandExec { import CatalogV2Implicits._ @@ -56,9 +56,11 @@ case class SetWriteDistributionAndOrderingExec( } orderBuilder.commit() - txn.updateProperties() - .set(WRITE_DISTRIBUTION_MODE, distributionMode.modeName()) - .commit() + distributionMode.foreach { mode => + txn.updateProperties() + .set(WRITE_DISTRIBUTION_MODE, mode.modeName()) + .commit() + } txn.commitTransaction() diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java index 77b7797fe192..b8547772da67 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java @@ -200,8 +200,7 @@ public void testSetWriteLocallyOrdered() { table.refresh(); - String distributionMode = table.properties().get(TableProperties.WRITE_DISTRIBUTION_MODE); - assertThat(distributionMode).as("Distribution mode must match").isEqualTo("none"); + assertThat(table.properties().containsKey(TableProperties.WRITE_DISTRIBUTION_MODE)).isFalse(); SortOrder expected = SortOrder.builderFor(table.schema()) @@ -213,6 +212,25 @@ public void testSetWriteLocallyOrdered() { assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); } + @TestTemplate + public void testSetWriteLocallyOrderedToPartitionedTable() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string) USING iceberg PARTITIONED BY (id)", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE LOCALLY ORDERED BY category DESC", tableName); + + table.refresh(); + + assertThat(table.properties().containsKey(TableProperties.WRITE_DISTRIBUTION_MODE)).isFalse(); + + SortOrder expected = + SortOrder.builderFor(table.schema()).withOrderId(1).desc("category").build(); + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); + } + @TestTemplate public void testSetWriteDistributedByWithSort() { sql( @@ -249,6 +267,13 @@ public void testSetWriteDistributedByWithLocalSort() { SortOrder expected = SortOrder.builderFor(table.schema()).withOrderId(1).asc("id").build(); assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); + + sql("ALTER TABLE %s WRITE LOCALLY ORDERED BY id", tableName); + + table.refresh(); + + String newDistributionMode = table.properties().get(TableProperties.WRITE_DISTRIBUTION_MODE); + assertThat(newDistributionMode).as("Distribution mode must match").isEqualTo(distributionMode); } @TestTemplate diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala index 0a0234cdfe34..7b599eb3da1d 100644 --- a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits case class SetWriteDistributionAndOrdering( table: Seq[String], - distributionMode: DistributionMode, + distributionMode: Option[DistributionMode], sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafCommand { import CatalogV2Implicits._ From 7738e1d7228474e36f661cfa1a15a2e8f8410bcd Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Sat, 26 Oct 2024 02:07:05 +0530 Subject: [PATCH 043/313] Spec: Fix table of content generation (#11067) --- format/spec.md | 94 +++++++++++++++++++++++++------------------------- 1 file changed, 47 insertions(+), 47 deletions(-) diff --git a/format/spec.md b/format/spec.md index 601cbcc3bc4e..6b80e876ed43 100644 --- a/format/spec.md +++ b/format/spec.md @@ -30,13 +30,13 @@ Versions 1 and 2 of the Iceberg spec are complete and adopted by the community. The format version number is incremented when new features are added that will break forward-compatibility---that is, when older readers would not read newer table features correctly. Tables may continue to be written with an older version of the spec to ensure compatibility by not using features that are not yet implemented by processing engines. -#### Version 1: Analytic Data Tables +### Version 1: Analytic Data Tables Version 1 of the Iceberg spec defines how to manage large analytic tables using immutable file formats: Parquet, Avro, and ORC. All version 1 data and metadata files are valid after upgrading a table to version 2. [Appendix E](#version-2) documents how to default version 2 fields when reading version 1 metadata. -#### Version 2: Row-level Deletes +### Version 2: Row-level Deletes Version 2 of the Iceberg spec adds row-level updates and deletes for analytic tables with immutable files. @@ -44,7 +44,7 @@ The primary change in version 2 adds delete files to encode rows that are delete In addition to row-level deletes, version 2 makes some requirements stricter for writers. The full set of changes are listed in [Appendix E](#version-2). -#### Version 3: Extended Types and Capabilities +### Version 3: Extended Types and Capabilities Version 3 of the Iceberg spec extends data types and existing metadata structures to add new capabilities: @@ -75,7 +75,7 @@ Data files in snapshots are tracked by one or more manifest files that contain a The manifests that make up a snapshot are stored in a manifest list file. Each manifest list stores metadata about manifests, including partition stats and data file counts. These stats are used to avoid reading manifests that are not required for an operation. -#### Optimistic Concurrency +### Optimistic Concurrency An atomic swap of one table metadata file for another provides the basis for serializable isolation. Readers use the snapshot that was current when they load the table metadata and are not affected by changes until they refresh and pick up a new metadata location. @@ -85,7 +85,7 @@ If the snapshot on which an update is based is no longer current, the writer mus The conditions required by a write to successfully commit determines the isolation level. Writers can select what to validate and can make different isolation guarantees. -#### Sequence Numbers +### Sequence Numbers The relative age of data and delete files relies on a sequence number that is assigned to every successful commit. When a snapshot is created for a commit, it is optimistically assigned the next sequence number, and it is written into the snapshot's metadata. If the commit fails and must be retried, the sequence number is reassigned and written into new snapshot metadata. @@ -94,7 +94,7 @@ All manifests, data files, and delete files created for a snapshot inherit the s Inheriting the sequence number from manifest metadata allows writing a new manifest once and reusing it in commit retries. To change a sequence number for a retry, only the manifest list must be rewritten -- which would be rewritten anyway with the latest set of manifests. -#### Row-level Deletes +### Row-level Deletes Row-level deletes are stored in delete files. @@ -106,7 +106,7 @@ There are two ways to encode a row-level delete: Like data files, delete files are tracked by partition. In general, a delete file must be applied to older data files with the same partition; see [Scan Planning](#scan-planning) for details. Column metrics can be used to determine whether a delete file's rows overlap the contents of a data file or a scan range. -#### File System Operations +### File System Operations Iceberg only requires that file systems support the following operations: @@ -121,9 +121,9 @@ Tables do not require random-access writes. Once written, data and metadata file Tables do not require rename, except for tables that use atomic rename to implement the commit operation for new metadata files. -# Specification +## Specification -### Terms +#### Terms * **Schema** -- Names and types of fields in a table. * **Partition spec** -- A definition of how partition values are derived from data fields. @@ -133,7 +133,7 @@ Tables do not require rename, except for tables that use atomic rename to implem * **Data file** -- A file that contains rows of a table. * **Delete file** -- A file that encodes rows of a table that are deleted by position or data values. -### Writer requirements +#### Writer requirements Some tables in this spec have columns that specify requirements for tables by version. These requirements are intended for writers when adding metadata files (including manifests files and manifest lists) to a table with the given version. @@ -158,19 +158,19 @@ Readers should be more permissive because v1 metadata files are allowed in v2 ta Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required fields that were not present in or were optional in prior versions may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. -### Writing data files +#### Writing data files All columns must be written to data files even if they introduce redundancy with metadata stored in manifest files (e.g. columns with identity partition transforms). Writing all columns provides a backup in case of corruption or bugs in the metadata layer. Writers are not allowed to commit files with a partition spec that contains a field with an unknown transform. -## Schemas and Data Types +### Schemas and Data Types A table's **schema** is a list of named columns. All data types are either primitives or nested types, which are maps, lists, or structs. A table schema is also a struct type. For the representations of these types in Avro, ORC, and Parquet file formats, see Appendix A. -### Nested Types +#### Nested Types A **`struct`** is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. Fields may have an optional comment or doc string. Fields can have [default values](#default-values). @@ -178,7 +178,7 @@ A **`list`** is a collection of values with some element type. The element field A **`map`** is a collection of key-value pairs with a key type and a value type. Both the key field and value field each have an integer id that is unique in the table schema. Map keys are required and map values can be either optional or required. Both map keys and map values may be any type, including nested types. -### Primitive Types +#### Primitive Types Supported primitive types are defined in the table below. Primitive types added after v1 have an "added by" version that is the first spec version in which the type is allowed. For example, nanosecond-precision timestamps are part of the v3 spec; using v3 types in v1 or v2 tables can break forward compatibility. @@ -211,7 +211,7 @@ Notes: For details on how to serialize a schema to JSON, see Appendix C. -### Default values +#### Default values Default values can be tracked for struct fields (both nested structs and the top-level schema's struct). There can be two defaults with a field: @@ -227,7 +227,7 @@ All columns of `unknown` type must default to null. Non-null values for `initial Default values are attributes of fields in schemas and serialized with fields in the JSON format. See [Appendix C](#appendix-c-json-serialization). -### Schema Evolution +#### Schema Evolution Schemas may be evolved by type promotion or adding, deleting, renaming, or reordering fields in structs (both nested structs and the top-level schema’s struct). @@ -275,7 +275,7 @@ Struct evolution requires the following rules for default values: * If a field value is missing from a struct's `write-default`, the field's `write-default` must be used for the field -#### Column Projection +##### Column Projection Columns in Iceberg data files are selected by field id. The table schema's column names and order may change after a data file is written, and projection must be done using field ids. @@ -307,7 +307,7 @@ Field mapping fields are constrained by the following rules: For details on serialization, see [Appendix C](#name-mapping-serialization). -### Identifier Field IDs +#### Identifier Field IDs A schema can optionally track the set of primitive fields that identify rows in a table, using the property `identifier-field-ids` (see JSON encoding in Appendix C). @@ -316,7 +316,7 @@ Two rows are the "same"---that is, the rows represent the same entity---if the i Identifier fields may be nested in structs but cannot be nested within maps or lists. Float, double, and optional fields cannot be used as identifier fields and a nested field cannot be used as an identifier field if it is nested in an optional struct, to avoid null values in identifiers. -### Reserved Field IDs +#### Reserved Field IDs Iceberg tables must not use field ids greater than 2147483447 (`Integer.MAX_VALUE - 200`). This id range is reserved for metadata columns that can be used in user data schemas, like the `_file` column that holds the file path in which a row was stored. @@ -335,7 +335,7 @@ The set of metadata columns is: | **`2147483543 _row_id`** | `long` | A unique long assigned when row-lineage is enabled, see [Row Lineage](#row-lineage) | | **`2147483542 _last_updated_sequence_number`** | `long` | The sequence number which last updated this row when row-lineage is enabled [Row Lineage](#row-lineage) | -### Row Lineage +#### Row Lineage In v3 and later, an Iceberg table can track row lineage fields for all newly created rows. Row lineage is enabled by setting the field `row-lineage` to true in the table's metadata. When enabled, engines must maintain the `next-row-id` table field and the following row-level fields when writing data files: @@ -347,7 +347,7 @@ These fields are assigned and updated by inheritance because the commit sequence When row lineage is enabled, new snapshots cannot include [Equality Deletes](#equality-delete-files). Row lineage is incompatible with equality deletes because lineage values must be maintained, but equality deletes are used to avoid reading existing data before writing changes. -#### Row lineage assignment +##### Row lineage assignment Row lineage fields are written when row lineage is enabled. When not enabled, row lineage fields (`_row_id` and `_last_updated_sequence_number`) must not be written to data files. The rest of this section applies when row lineage is enabled. @@ -368,7 +368,7 @@ When an existing row is moved to a different data file for any reason, writers a 3. If the write has not modified the row, the existing non-null `_last_updated_sequence_number` value must be copied to the new data file -#### Row lineage example +##### Row lineage example This example demonstrates how `_row_id` and `_last_updated_sequence_number` are assigned for a snapshot when row lineage is enabled. This starts with a table with row lineage enabled and a `next-row-id` of 1000. @@ -409,7 +409,7 @@ Files `data2` and `data3` are written with `null` for `first_row_id` and are ass When the new snapshot is committed, the table's `next-row-id` must also be updated (even if the new snapshot is not in the main branch). Because 225 rows were added (`added1`: 100 + `added2`: 0 + `added3`: 125), the new value is 1,000 + 225 = 1,225: -### Enabling Row Lineage for Non-empty Tables +##### Enabling Row Lineage for Non-empty Tables Any snapshot without the field `first-row-id` does not have any lineage information and values for `_row_id` and `_last_updated_sequence_number` cannot be assigned accurately. @@ -419,7 +419,7 @@ null should be explicitly written. After this point, rows are treated as if they and assigned `row_id` and `_last_updated_sequence_number` as if they were new rows. -## Partitioning +### Partitioning Data files are stored in manifests with a tuple of partition values that are used in scans to filter out files that cannot contain records that match the scan’s filter predicate. Partition values for a data file must be the same for all records stored in the data file. (Manifests store data files from any partition, as long as the partition spec is the same for the data files.) @@ -440,7 +440,7 @@ Two partition specs are considered equivalent with each other if they have the s Partition field IDs must be reused if an existing partition spec contains an equivalent field. -### Partition Transforms +#### Partition Transforms | Transform name | Description | Source types | Result type | |-------------------|--------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|-------------| @@ -458,7 +458,7 @@ All transforms must return `null` for a `null` input value. The `void` transform may be used to replace the transform in an existing partition field so that the field is effectively dropped in v1 tables. See partition evolution below. -### Bucket Transform Details +#### Bucket Transform Details Bucket partition transforms use a 32-bit hash of the source value. The 32-bit hash implementation is the 32-bit Murmur3 hash, x86 variant, seeded with 0. @@ -475,7 +475,7 @@ Notes: For hash function details by type, see Appendix B. -### Truncate Transform Details +#### Truncate Transform Details | **Type** | **Config** | **Truncate specification** | **Examples** | |---------------|-----------------------|------------------------------------------------------------------|----------------------------------| @@ -493,7 +493,7 @@ Notes: 4. In contrast to strings, binary values do not have an assumed encoding and are truncated to `L` bytes. -### Partition Evolution +#### Partition Evolution Table partitioning can be evolved by adding, removing, renaming, or reordering partition spec fields. @@ -510,7 +510,7 @@ In v1, partition field IDs were not tracked, but were assigned sequentially star 3. Only add partition fields at the end of the previous partition spec -## Sorting +### Sorting Users can sort their data within partitions by columns to gain performance. The information on how the data is sorted can be declared per data or delete file, by a **sort order**. @@ -530,7 +530,7 @@ Sorting floating-point numbers should produce the following behavior: `-NaN` < ` A data or delete file is associated with a sort order by the sort order's id within [a manifest](#manifests). Therefore, the table must declare all the sort orders for lookup. A table could also be configured with a default sort order id, indicating how the new data should be sorted by default. Writers should use this default sort order to sort the data on write, but are not required to if the default order is prohibitively expensive, as it would be for streaming writes. -## Manifests +### Manifests A manifest is an immutable Avro file that lists data files or delete files, along with each file’s partition data tuple, metrics, and tracking information. One or more manifest files are used to store a [snapshot](#snapshots), which tracks all of the files in a table at some point in time. Manifests are tracked by a [manifest list](#manifest-lists) for each table snapshot. @@ -598,7 +598,7 @@ The `partition` struct stores the tuple of partition values for each file. Its t The column metrics maps are used when filtering to select both data and delete files. For delete files, the metrics must store bounds and counts for all deleted rows, or must be omitted. Storing metrics for deleted rows ensures that the values can be used during job planning to find delete files that must be merged during a scan. -### Manifest Entry Fields +#### Manifest Entry Fields The manifest entry fields are used to keep track of the snapshot in which files were added or logically deleted. The `data_file` struct is nested inside of the manifest entry so that it can be easily passed to job planning without the manifest entry fields. @@ -616,7 +616,7 @@ Notes: 1. Technically, data files can be deleted when the last snapshot that contains the file as “live” data is garbage collected. But this is harder to detect and requires finding the diff of multiple snapshots. It is easier to track what files are deleted in a snapshot and delete them when that snapshot expires. It is not recommended to add a deleted file back to a table. Adding a deleted file can lead to edge cases where incremental deletes can break table snapshots. 2. Manifest list files are required in v2, so that the `sequence_number` and `snapshot_id` to inherit are always available. -### Sequence Number Inheritance +#### Sequence Number Inheritance Manifests track the sequence number when a data or delete file was added to the table. @@ -629,7 +629,7 @@ Inheriting sequence numbers through the metadata tree allows writing a new manif When reading v1 manifests with no sequence number column, sequence numbers for all files must default to 0. -### First Row ID Inheritance +#### First Row ID Inheritance Row ID inheritance is used when row lineage is enabled. When not enabled, a data file's `first_row_id` must always be set to `null`. The rest of this section applies when row lineage is enabled. @@ -639,7 +639,7 @@ When reading, the `first_row_id` is assigned by replacing `null` with the manife The `first_row_id` is only inherited for added data files. The inherited value must be written into the data file metadata for existing and deleted entries. The value of `first_row_id` for delete files is always `null`. -## Snapshots +### Snapshots A snapshot consists of the following fields: @@ -673,7 +673,7 @@ Manifests for a snapshot are tracked by a manifest list. Valid snapshots are stored as a list in table metadata. For serialization, see Appendix C. -### Snapshot Row IDs +#### Snapshot Row IDs When row lineage is not enabled, `first-row-id` must be omitted. The rest of this section applies when row lineage is enabled. @@ -811,13 +811,13 @@ When expiring snapshots, retention policies in table and snapshot references are 2. The snapshot is not one of the first `min-snapshots-to-keep` in the branch (including the branch's referenced snapshot) 5. Expire any snapshot not in the set of snapshots to retain. -## Table Metadata +### Table Metadata Table metadata is stored as JSON. Each table metadata change creates a new table metadata file that is committed by an atomic operation. This operation is used to ensure that a new version of table metadata replaces the version on which it was based. This produces a linear history of table versions and ensures that concurrent writes are not lost. The atomic operation used to commit metadata depends on how tables are tracked and is not standardized by this spec. See the sections below for examples. -### Table Metadata Fields +#### Table Metadata Fields Table metadata consists of the following fields: @@ -853,7 +853,7 @@ For serialization details, see Appendix C. When a new snapshot is added, the table's `next-row-id` should be updated to the previous `next-row-id` plus the sum of `record_count` for all data files added in the snapshot (this is also equal to the sum of `added_rows_count` for all manifests added in the snapshot). This ensures that `next-row-id` is always higher than any assigned row ID in the table. -### Table Statistics +#### Table Statistics Table statistics files are valid [Puffin files](puffin-spec.md). Statistics are informational. A reader can choose to ignore statistics information. Statistics support is not required to read the table correctly. A table can contain @@ -881,7 +881,7 @@ Blob metadata is a struct with the following fields: | _optional_ | _optional_ | **`properties`** | `map` | Additional properties associated with the statistic. Subset of Blob properties in the Puffin file. | -### Partition Statistics +#### Partition Statistics Partition statistics files are based on [partition statistics file spec](#partition-statistics-file). Partition statistics are not required for reading or planning and readers may ignore them. @@ -897,7 +897,7 @@ Partition statistics file must be registered in the table metadata file to be co | _required_ | _required_ | **`statistics-path`** | `string` | Path of the partition statistics file. See [Partition statistics file](#partition-statistics-file). | | _required_ | _required_ | **`file-size-in-bytes`** | `long` | Size of the partition statistics file. | -#### Partition Statistics File +##### Partition Statistics File Statistics information for each unique partition tuple is stored as a row in any of the data file format of the table (for example, Parquet or ORC). These rows must be sorted (in ascending manner with NULL FIRST) by `partition` field to optimize filtering rows while scanning. @@ -934,7 +934,7 @@ The unified partition type looks like `Struct`. and then the table has evolved into `spec#1` which has just one field `{field#2}`. The unified partition type looks like `Struct`. -## Commit Conflict Resolution and Retry +### Commit Conflict Resolution and Retry When two commits happen at the same time and are based on the same version, only one commit will succeed. In most cases, the failed commit can be applied to the new current version of table metadata and retried. Updates verify the conditions under which they can be applied to a new version and retry if those conditions are met. @@ -944,7 +944,7 @@ When two commits happen at the same time and are based on the same version, only * Table schema updates and partition spec changes must validate that the schema has not changed between the base version and the current version. -### File System Tables +#### File System Tables _Note: This file system based scheme to commit a metadata file is **deprecated** and will be removed in version 4 of this spec. The scheme is **unsafe** in object stores and local file systems._ @@ -963,7 +963,7 @@ Notes: 1. The file system table scheme is implemented in [HadoopTableOperations](../javadoc/{{ icebergVersion }}/index.html?org/apache/iceberg/hadoop/HadoopTableOperations.html). -### Metastore Tables +#### Metastore Tables The atomic swap needed to commit new versions of table metadata can be implemented by storing a pointer in a metastore or database that is updated with a check-and-put operation [1]. The check-and-put validates that the version of the table that a write is based on is still current and then makes the new metadata from the write the current version. @@ -980,7 +980,7 @@ Notes: 1. The metastore table scheme is partly implemented in [BaseMetastoreTableOperations](../javadoc/{{ icebergVersion }}/index.html?org/apache/iceberg/BaseMetastoreTableOperations.html). -## Delete Formats +### Delete Formats This section details how to encode row-level deletes in Iceberg delete files. Row-level deletes are not supported in v1. @@ -991,7 +991,7 @@ Row-level delete files are tracked by manifests, like data files. A separate set Both position and equality deletes allow encoding deleted row values with a delete. This can be used to reconstruct a stream of changes to a table. -### Position Delete Files +#### Position Delete Files Position-based delete files identify deleted rows by file and position in one or more data files, and may optionally contain the deleted row. @@ -1016,7 +1016,7 @@ The rows in the delete file must be sorted by `file_path` then `pos` to optimize * Sorting by `file_path` allows filter pushdown by file in columnar storage formats. * Sorting by `pos` allows filtering rows while scanning, to avoid keeping deletes in memory. -### Equality Delete Files +#### Equality Delete Files Equality delete files identify deleted rows in a collection of data files by one or more column values, and may optionally contain additional columns of the deleted row. @@ -1068,7 +1068,7 @@ equality_ids=[1, 2] If a delete column in an equality delete file is later dropped from the table, it must still be used when applying the equality deletes. If a column was added to a table and later used as a delete column in an equality delete file, the column value is read for older data files using normal projection rules (defaults to `null`). -### Delete File Stats +#### Delete File Stats Manifests hold the same statistics for delete files and data files. For delete files, the metrics describe the values that were deleted. From 9ecd97bf4538ca94276fb019c2ec477d28e4bf7c Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Fri, 25 Oct 2024 13:39:48 -0700 Subject: [PATCH 044/313] [KafkaConnect] Fix RecordConverter for UUID and Fixed Types (#11346) --- .../iceberg/connect/data/RecordConverter.java | 25 ++++++++++++--- .../connect/data/RecordConverterTest.java | 32 ++++++++++++++++--- 2 files changed, 47 insertions(+), 10 deletions(-) diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java index a312e69001c3..1a57a6444870 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java @@ -41,6 +41,7 @@ import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -60,7 +61,9 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.types.Types.TimestampType; +import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.UUIDUtil; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; @@ -130,8 +133,9 @@ private Object convertValue( case UUID: return convertUUID(value); case BINARY: - case FIXED: return convertBase64Binary(value); + case FIXED: + return ByteBuffers.toByteArray(convertBase64Binary(value)); case DATE: return convertDateValue(value); case TIME: @@ -390,13 +394,24 @@ protected String convertString(Object value) { throw new IllegalArgumentException("Cannot convert to string: " + value.getClass().getName()); } - protected UUID convertUUID(Object value) { + protected Object convertUUID(Object value) { + UUID uuid; if (value instanceof String) { - return UUID.fromString((String) value); + uuid = UUID.fromString((String) value); } else if (value instanceof UUID) { - return (UUID) value; + uuid = (UUID) value; + } else { + throw new IllegalArgumentException("Cannot convert to UUID: " + value.getClass().getName()); + } + + if (FileFormat.PARQUET + .name() + .toLowerCase(Locale.ROOT) + .equals(config.writeProps().get(TableProperties.DEFAULT_FILE_FORMAT))) { + return UUIDUtil.convert(uuid); + } else { + return uuid; } - throw new IllegalArgumentException("Cannot convert to UUID: " + value.getClass().getName()); } protected ByteBuffer convertBase64Binary(Object value) { diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java index b494a9da85d3..47ee76eade15 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java @@ -37,9 +37,11 @@ import java.util.Collection; import java.util.Date; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.UUID; import java.util.function.Function; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.connect.IcebergSinkConfig; @@ -72,6 +74,7 @@ import org.apache.iceberg.types.Types.TimeType; import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.types.Types.UUIDType; +import org.apache.iceberg.util.UUIDUtil; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; @@ -221,6 +224,25 @@ public void testMapConvert() { assertRecordValues(record); } + @Test + public void testUUIDConversionWithParquet() { + Table table = mock(Table.class); + when(table.schema()) + .thenReturn(new org.apache.iceberg.Schema(NestedField.required(1, "uuid", UUIDType.get()))); + when(config.writeProps()) + .thenReturn( + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.PARQUET.name().toLowerCase(Locale.ROOT))); + + RecordConverter converter = new RecordConverter(table, config); + Map data = + ImmutableMap.builder().put("uuid", UUID_VAL.toString()).build(); + + Record record = converter.convert(data); + assertThat(record.getField("uuid")).isEqualTo(UUIDUtil.convert(UUID_VAL)); + } + @Test public void testNestedMapConvert() { Table table = mock(Table.class); @@ -859,7 +881,7 @@ public void testEvolveTypeDetectionStructNested() { assertThat(updateMap.get("st.ff").type()).isInstanceOf(DoubleType.class); } - private Map createMapData() { + public static Map createMapData() { return ImmutableMap.builder() .put("i", 1) .put("l", 2L) @@ -898,8 +920,8 @@ private Struct createStructData() { .put("s", STR_VAL) .put("b", true) .put("u", UUID_VAL.toString()) - .put("f", BYTES_VAL.array()) - .put("bi", BYTES_VAL.array()) + .put("f", BYTES_VAL) + .put("bi", BYTES_VAL) .put("li", LIST_VAL) .put("ma", MAP_VAL); } @@ -921,11 +943,11 @@ private void assertRecordValues(Record record) { assertThat(rec.getField("dec")).isEqualTo(DEC_VAL); assertThat(rec.getField("s")).isEqualTo(STR_VAL); assertThat(rec.getField("b")).isEqualTo(true); - assertThat(rec.getField("u")).isEqualTo(UUID_VAL); - assertThat(rec.getField("f")).isEqualTo(BYTES_VAL); + assertThat(rec.getField("f")).isEqualTo(BYTES_VAL.array()); assertThat(rec.getField("bi")).isEqualTo(BYTES_VAL); assertThat(rec.getField("li")).isEqualTo(LIST_VAL); assertThat(rec.getField("ma")).isEqualTo(MAP_VAL); + assertThat(rec.getField("u")).isEqualTo(UUID_VAL); } private void assertNestedRecordValues(Record record) { From 7ad11b2df1a266d29f9e4f6bb5b499cb68c0afb7 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Fri, 25 Oct 2024 17:23:44 -0400 Subject: [PATCH 045/313] Core: Snapshot `summary` map must have `operation` key (#11354) Co-authored-by: Eduard Tudenhoefner --- .../org/apache/iceberg/SnapshotParser.java | 5 +- .../org/apache/iceberg/TestSnapshotJson.java | 77 +++++++++++++++++++ 2 files changed, 79 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotParser.java b/core/src/main/java/org/apache/iceberg/SnapshotParser.java index bc5ef6094695..41b8e1499c0a 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotParser.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotParser.java @@ -129,13 +129,12 @@ static Snapshot fromJson(JsonNode node) { "Cannot parse summary from non-object value: %s", sNode); + operation = JsonUtil.getString(OPERATION, sNode); ImmutableMap.Builder builder = ImmutableMap.builder(); Iterator fields = sNode.fieldNames(); while (fields.hasNext()) { String field = fields.next(); - if (field.equals(OPERATION)) { - operation = JsonUtil.getString(OPERATION, sNode); - } else { + if (!field.equals(OPERATION)) { builder.put(field, JsonUtil.getString(field, sNode)); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index ee1239074997..7fff5c5dddd9 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -20,11 +20,16 @@ import static org.apache.iceberg.Files.localInput; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.File; import java.io.IOException; import java.nio.file.Path; import java.util.List; +import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -35,6 +40,56 @@ public class TestSnapshotJson { public TableOperations ops = new LocalTableOperations(temp); + @Test + public void testToJsonWithoutOperation() throws IOException { + int snapshotId = 23; + Long parentId = null; + String manifestList = createManifestListWithManifestFiles(snapshotId, parentId); + + Snapshot expected = + new BaseSnapshot( + 0, snapshotId, parentId, System.currentTimeMillis(), null, null, 1, manifestList); + String json = SnapshotParser.toJson(expected); + + // Assert that summary field is not present in the JSON + assertThat(new ObjectMapper().readTree(json)).anyMatch(node -> !node.has("summary")); + } + + @Test + public void testToJsonWithOperation() throws IOException { + long parentId = 1; + long id = 2; + + String manifestList = createManifestListWithManifestFiles(id, parentId); + + Snapshot expected = + new BaseSnapshot( + 0, + id, + parentId, + System.currentTimeMillis(), + DataOperations.REPLACE, + ImmutableMap.of("files-added", "4", "files-deleted", "100"), + 3, + manifestList); + Map expectedSummary = + ImmutableMap.builder() + .putAll(expected.summary()) + .put("operation", expected.operation()) // operation should be part of the summary + .build(); + + String json = SnapshotParser.toJson(expected); + ObjectMapper objectMapper = new ObjectMapper(); + JsonNode jsonNode = objectMapper.readTree(json); + + assertThat(jsonNode.get("summary")).isNotNull(); + + Map actualSummary = + objectMapper.convertValue( + jsonNode.get("summary"), new TypeReference>() {}); + assertThat(actualSummary).isEqualTo(expectedSummary); + } + @Test public void testJsonConversion() throws IOException { int snapshotId = 23; @@ -159,6 +214,28 @@ public void testJsonConversionWithV1Manifests() { assertThat(snapshot.schemaId()).isEqualTo(expected.schemaId()); } + @Test + public void testJsonConversionSummaryWithoutOperationFails() { + String json = + String.format( + "{\n" + + " \"snapshot-id\" : 2,\n" + + " \"parent-snapshot-id\" : 1,\n" + + " \"timestamp-ms\" : %s,\n" + + " \"summary\" : {\n" + + " \"files-added\" : \"4\",\n" + + " \"files-deleted\" : \"100\"\n" + + " },\n" + + " \"manifests\" : [ \"/tmp/manifest1.avro\", \"/tmp/manifest2.avro\" ],\n" + + " \"schema-id\" : 3\n" + + "}", + System.currentTimeMillis()); + + assertThatThrownBy(() -> SnapshotParser.fromJson(json)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: operation"); + } + private String createManifestListWithManifestFiles(long snapshotId, Long parentSnapshotId) throws IOException { File manifestList = File.createTempFile("manifests", null, temp.toFile()); From 2b55fef7cc2a249d864ac26d85a4923313d96a59 Mon Sep 17 00:00:00 2001 From: erik-grepr Date: Fri, 25 Oct 2024 22:57:24 -0700 Subject: [PATCH 046/313] Core: Update TableMetadataParser to ensure all streams closed (#11220) * Update TableMetadataParser to close streams * Fix OutputStreamWriter * Update TableMetadataParser.java * spotlessApply --- .../java/org/apache/iceberg/TableMetadataParser.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index 8bda184142cd..5abfb74974f7 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -122,9 +122,9 @@ public static void write(TableMetadata metadata, OutputFile outputFile) { public static void internalWrite( TableMetadata metadata, OutputFile outputFile, boolean overwrite) { boolean isGzip = Codec.fromFileName(outputFile.location()) == Codec.GZIP; - OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create(); - try (OutputStream ou = isGzip ? new GZIPOutputStream(stream) : stream; - OutputStreamWriter writer = new OutputStreamWriter(ou, StandardCharsets.UTF_8)) { + try (OutputStream os = overwrite ? outputFile.createOrOverwrite() : outputFile.create(); + OutputStream gos = isGzip ? new GZIPOutputStream(os) : os; + OutputStreamWriter writer = new OutputStreamWriter(gos, StandardCharsets.UTF_8)) { JsonGenerator generator = JsonUtil.factory().createGenerator(writer); generator.useDefaultPrettyPrinter(); toJson(metadata, generator); @@ -275,9 +275,9 @@ public static TableMetadata read(FileIO io, String path) { public static TableMetadata read(FileIO io, InputFile file) { Codec codec = Codec.fromFileName(file.location()); - try (InputStream is = - codec == Codec.GZIP ? new GZIPInputStream(file.newStream()) : file.newStream()) { - return fromJson(file, JsonUtil.mapper().readValue(is, JsonNode.class)); + try (InputStream is = file.newStream(); + InputStream gis = codec == Codec.GZIP ? new GZIPInputStream(is) : is) { + return fromJson(file, JsonUtil.mapper().readValue(gis, JsonNode.class)); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to read file: %s", file); } From e3bbcac8524f1835fc8737ec6af26de3177b4e01 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 28 Oct 2024 17:04:24 +0800 Subject: [PATCH 047/313] Build: Bump Spark 3.4 to 3.4.4 (#11366) --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 2a01ea0b99f8..39d2b2e97b04 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -82,7 +82,7 @@ scala-collection-compat = "2.12.0" slf4j = "2.0.16" snowflake-jdbc = "3.19.0" spark-hive33 = "3.3.4" -spark-hive34 = "3.4.3" +spark-hive34 = "3.4.4" spark-hive35 = "3.5.2" spring-boot = "2.7.18" spring-web = "5.3.39" From b4d178fa0502159ec087b1fced1747ee181fd4e3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 10:05:32 +0100 Subject: [PATCH 048/313] Build: Bump junit from 5.11.1 to 5.11.3 (#11401) Bumps `junit` from 5.11.1 to 5.11.3. Updates `org.junit.jupiter:junit-jupiter` from 5.11.1 to 5.11.3 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/compare/r5.11.1...r5.11.3) Updates `org.junit.jupiter:junit-jupiter-engine` from 5.11.1 to 5.11.3 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/compare/r5.11.1...r5.11.3) Updates `org.junit.vintage:junit-vintage-engine` from 5.11.1 to 5.11.3 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/compare/r5.11.1...r5.11.3) --- updated-dependencies: - dependency-name: org.junit.jupiter:junit-jupiter dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.jupiter:junit-jupiter-engine dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.vintage:junit-vintage-engine dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 39d2b2e97b04..e3d1ea6ab1be 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -63,7 +63,7 @@ jakarta-servlet-api = "6.1.0" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "11.0.24" -junit = "5.11.1" +junit = "5.11.3" junit-platform = "1.11.2" kafka = "3.8.0" kryo-shaded = "4.0.3" From 9565b9c401e87c9d6c31d91e44b439c1cb16d123 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 10:06:12 +0100 Subject: [PATCH 049/313] Build: Bump software.amazon.awssdk:bom from 2.28.26 to 2.29.1 (#11400) Bumps software.amazon.awssdk:bom from 2.28.26 to 2.29.1. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index e3d1ea6ab1be..932c78f85bbe 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.28.26" +awssdk-bom = "2.29.1" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.2.0" caffeine = "2.9.3" From 681b09ddc95e66b056aa7954c6606c5ff329cb24 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Mon, 28 Oct 2024 10:13:50 +0100 Subject: [PATCH 050/313] Core: Move Javadoc about commit retries to SnapshotProducer (#10995) --- .../java/org/apache/iceberg/FastAppend.java | 8 +----- .../java/org/apache/iceberg/MergeAppend.java | 8 +----- .../org/apache/iceberg/SnapshotProducer.java | 6 +++++ .../org/apache/iceberg/StreamingDelete.java | 8 +----- .../org/apache/iceberg/TestFastAppend.java | 25 +++++++++++++++++++ 5 files changed, 34 insertions(+), 21 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 1b6e1b3b52bc..1e2f6fe0d90d 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -24,7 +24,6 @@ import java.util.Set; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.events.CreateSnapshotEvent; -import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -32,12 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.DataFileSet; -/** - * {@link AppendFiles Append} implementation that adds a new manifest file for the write. - * - *

This implementation will attempt to commit 5 times before throwing {@link - * CommitFailedException}. - */ +/** {@link AppendFiles Append} implementation that adds a new manifest file for the write. */ class FastAppend extends SnapshotProducer implements AppendFiles { private final String tableName; private final TableOperations ops; diff --git a/core/src/main/java/org/apache/iceberg/MergeAppend.java b/core/src/main/java/org/apache/iceberg/MergeAppend.java index 3ef553ba7832..231ad8cc5d06 100644 --- a/core/src/main/java/org/apache/iceberg/MergeAppend.java +++ b/core/src/main/java/org/apache/iceberg/MergeAppend.java @@ -18,15 +18,9 @@ */ package org.apache.iceberg; -import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** - * Append implementation that produces a minimal number of manifest files. - * - *

This implementation will attempt to commit 5 times before throwing {@link - * CommitFailedException}. - */ +/** {@link AppendFiles Append} implementation that produces a minimal number of manifest files. */ class MergeAppend extends MergingSnapshotProducer implements AppendFiles { MergeAppend(String tableName, TableOperations ops) { super(tableName, ops); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 33114baa641d..89f9eab7192a 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -80,6 +80,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Keeps common functionality to create a new snapshot. + * + *

The number of attempted commits is controlled by {@link TableProperties#COMMIT_NUM_RETRIES} + * and {@link TableProperties#COMMIT_NUM_RETRIES_DEFAULT} properties. + */ @SuppressWarnings("UnnecessaryAnonymousClass") abstract class SnapshotProducer implements SnapshotUpdate { private static final Logger LOG = LoggerFactory.getLogger(SnapshotProducer.class); diff --git a/core/src/main/java/org/apache/iceberg/StreamingDelete.java b/core/src/main/java/org/apache/iceberg/StreamingDelete.java index df5a11bf31c5..81621164e4af 100644 --- a/core/src/main/java/org/apache/iceberg/StreamingDelete.java +++ b/core/src/main/java/org/apache/iceberg/StreamingDelete.java @@ -18,15 +18,9 @@ */ package org.apache.iceberg; -import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.expressions.Expression; -/** - * {@link DeleteFiles Delete} implementation that avoids loading full manifests in memory. - * - *

This implementation will attempt to commit 5 times before throwing {@link - * CommitFailedException}. - */ +/** {@link DeleteFiles Delete} implementation that avoids loading full manifests in memory. */ public class StreamingDelete extends MergingSnapshotProducer implements DeleteFiles { private boolean validateFilesToDeleteExist = false; diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 7a93b9988730..8da9cb0e33ef 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -271,6 +271,31 @@ public void testFailure() { assertThat(new File(newManifest.path())).doesNotExist(); } + @TestTemplate + public void testIncreaseNumRetries() { + TestTables.TestTableOperations ops = table.ops(); + ops.failCommits(TableProperties.COMMIT_NUM_RETRIES_DEFAULT + 1); + + AppendFiles append = table.newFastAppend().appendFile(FILE_B); + + // Default number of retries results in a failed commit + assertThatThrownBy(append::commit) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); + + // After increasing the number of retries the commit succeeds + table + .updateProperties() + .set( + TableProperties.COMMIT_NUM_RETRIES, + String.valueOf(TableProperties.COMMIT_NUM_RETRIES_DEFAULT + 1)) + .commit(); + + append.commit(); + + validateSnapshot(null, readMetadata().currentSnapshot(), FILE_B); + } + @TestTemplate public void testAppendManifestCleanup() throws IOException { // inject 5 failures From 9fc9c052b592d7e79b2900eccf0459e1951d9e60 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 11:23:45 +0100 Subject: [PATCH 051/313] Build: Bump junit-platform from 1.11.2 to 1.11.3 (#11402) Bumps `junit-platform` from 1.11.2 to 1.11.3. Updates `org.junit.platform:junit-platform-suite-api` from 1.11.2 to 1.11.3 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/commits) Updates `org.junit.platform:junit-platform-suite-engine` from 1.11.2 to 1.11.3 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/commits) --- updated-dependencies: - dependency-name: org.junit.platform:junit-platform-suite-api dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.platform:junit-platform-suite-engine dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 932c78f85bbe..3ae93c183b3f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -64,7 +64,7 @@ jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "11.0.24" junit = "5.11.3" -junit-platform = "1.11.2" +junit-platform = "1.11.3" kafka = "3.8.0" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" From 68a7102073d53e81a082cc56b6f5dac6d5f436ae Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 12:02:34 +0100 Subject: [PATCH 052/313] Build: Bump org.xerial:sqlite-jdbc from 3.46.1.3 to 3.47.0.0 (#11407) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.46.1.3 to 3.47.0.0. - [Release notes](https://github.com/xerial/sqlite-jdbc/releases) - [Changelog](https://github.com/xerial/sqlite-jdbc/blob/master/CHANGELOG) - [Commits](https://github.com/xerial/sqlite-jdbc/compare/3.46.1.3...3.47.0.0) --- updated-dependencies: - dependency-name: org.xerial:sqlite-jdbc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 3ae93c183b3f..67f50a120cee 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -86,7 +86,7 @@ spark-hive34 = "3.4.4" spark-hive35 = "3.5.2" spring-boot = "2.7.18" spring-web = "5.3.39" -sqlite-jdbc = "3.46.1.3" +sqlite-jdbc = "3.47.0.0" testcontainers = "1.20.2" tez010 = "0.10.4" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 48acaadcf94f43572f11b7b589d1fb3857fc6b9d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 12:03:10 +0100 Subject: [PATCH 053/313] Build: Bump software.amazon.s3.accessgrants:aws-s3-accessgrants-java-plugin (#11405) Bumps [software.amazon.s3.accessgrants:aws-s3-accessgrants-java-plugin](https://github.com/aws/aws-s3-accessgrants-plugin-java-v2) from 2.2.0 to 2.3.0. - [Changelog](https://github.com/aws/aws-s3-accessgrants-plugin-java-v2/blob/main/CHANGELOG.md) - [Commits](https://github.com/aws/aws-s3-accessgrants-plugin-java-v2/commits) --- updated-dependencies: - dependency-name: software.amazon.s3.accessgrants:aws-s3-accessgrants-java-plugin dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 67f50a120cee..61d8bb03c38f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -31,7 +31,7 @@ assertj-core = "3.26.3" awaitility = "4.2.2" awssdk-bom = "2.29.1" azuresdk-bom = "1.2.25" -awssdk-s3accessgrants = "2.2.0" +awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" calcite = "1.10.0" datasketches = "6.1.1" From c3191eecd90cd0b33bb11b77ea2ebd4c35f5e751 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 14:15:42 +0100 Subject: [PATCH 054/313] Build: Bump net.snowflake:snowflake-jdbc from 3.19.0 to 3.19.1 (#11406) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.19.0 to 3.19.1. - [Release notes](https://github.com/snowflakedb/snowflake-jdbc/releases) - [Changelog](https://github.com/snowflakedb/snowflake-jdbc/blob/master/CHANGELOG.rst) - [Commits](https://github.com/snowflakedb/snowflake-jdbc/compare/v3.19.0...v3.19.1) --- updated-dependencies: - dependency-name: net.snowflake:snowflake-jdbc dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 61d8bb03c38f..aa3cd7b46355 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -80,7 +80,7 @@ pig = "0.17.0" roaringbitmap = "1.3.0" scala-collection-compat = "2.12.0" slf4j = "2.0.16" -snowflake-jdbc = "3.19.0" +snowflake-jdbc = "3.19.1" spark-hive33 = "3.3.4" spark-hive34 = "3.4.4" spark-hive35 = "3.5.2" From 6e911e0ea34499333c92406784b047402884bc54 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 14:16:03 +0100 Subject: [PATCH 055/313] Build: Bump testcontainers from 1.20.2 to 1.20.3 (#11404) Bumps `testcontainers` from 1.20.2 to 1.20.3. Updates `org.testcontainers:testcontainers` from 1.20.2 to 1.20.3 - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/1.20.2...1.20.3) Updates `org.testcontainers:junit-jupiter` from 1.20.2 to 1.20.3 - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/1.20.2...1.20.3) Updates `org.testcontainers:minio` from 1.20.2 to 1.20.3 - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/1.20.2...1.20.3) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.testcontainers:junit-jupiter dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.testcontainers:minio dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index aa3cd7b46355..ede2bed32629 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -87,7 +87,7 @@ spark-hive35 = "3.5.2" spring-boot = "2.7.18" spring-web = "5.3.39" sqlite-jdbc = "3.47.0.0" -testcontainers = "1.20.2" +testcontainers = "1.20.3" tez010 = "0.10.4" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From d10e67d422f2b58ed13bcf826bc95fe1e5acc0f4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 14:16:44 +0100 Subject: [PATCH 056/313] Build: Bump com.google.errorprone:error_prone_annotations (#11403) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.34.0 to 2.35.1. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.34.0...v2.35.1) --- updated-dependencies: - dependency-name: com.google.errorprone:error_prone_annotations dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index ede2bed32629..55aa97c350a9 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -38,7 +38,7 @@ datasketches = "6.1.1" delta-standalone = "3.2.1" delta-spark = "3.2.1" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.34.0" +errorprone-annotations = "2.35.1" failsafe = "3.3.2" findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} From b4b0bdc543078c55301da6e4b29b81f6e1329a49 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 14:18:50 +0100 Subject: [PATCH 057/313] Build: Bump mkdocs-macros-plugin from 1.2.0 to 1.3.7 (#11399) Bumps [mkdocs-macros-plugin](https://github.com/fralau/mkdocs_macros_plugin) from 1.2.0 to 1.3.7. - [Release notes](https://github.com/fralau/mkdocs_macros_plugin/releases) - [Changelog](https://github.com/fralau/mkdocs-macros-plugin/blob/master/CHANGELOG.md) - [Commits](https://github.com/fralau/mkdocs_macros_plugin/compare/v1.2.0...v1.3.7) --- updated-dependencies: - dependency-name: mkdocs-macros-plugin dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index a7db0af729d0..41f982b5dccb 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -16,7 +16,7 @@ # under the License. mkdocs-awesome-pages-plugin==2.9.3 -mkdocs-macros-plugin==1.2.0 +mkdocs-macros-plugin==1.3.7 mkdocs-material==9.5.39 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix From 696204392c97940e9d8fded77a43fcabc15e097e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 14:20:31 +0100 Subject: [PATCH 058/313] Build: Bump mkdocs-material from 9.5.39 to 9.5.42 (#11398) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.39 to 9.5.42. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.5.39...9.5.42) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 41f982b5dccb..5ace10851aa5 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.3.7 -mkdocs-material==9.5.39 +mkdocs-material==9.5.42 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.1 From 86a8560315432d6c63af8f6bc1379618d7b432cf Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 29 Oct 2024 00:42:58 +0800 Subject: [PATCH 059/313] Flink: Fix disabling flaky range distribution bucketing tests (#11410) --- .../sink/TestFlinkIcebergSinkRangeDistributionBucketing.java | 4 ++-- .../sink/TestFlinkIcebergSinkRangeDistributionBucketing.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java index 084c66317e2b..a5799288b5e3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -58,9 +58,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Ignore; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.RegisterExtension; @@ -78,7 +78,7 @@ * */ @Timeout(value = 30) -@Ignore // https://github.com/apache/iceberg/pull/11305#issuecomment-2415207097 +@Disabled // https://github.com/apache/iceberg/pull/11305#issuecomment-2415207097 public class TestFlinkIcebergSinkRangeDistributionBucketing { private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = new Configuration() diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java index 084c66317e2b..a5799288b5e3 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -58,9 +58,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Ignore; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.RegisterExtension; @@ -78,7 +78,7 @@ * */ @Timeout(value = 30) -@Ignore // https://github.com/apache/iceberg/pull/11305#issuecomment-2415207097 +@Disabled // https://github.com/apache/iceberg/pull/11305#issuecomment-2415207097 public class TestFlinkIcebergSinkRangeDistributionBucketing { private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = new Configuration() From a6503f573c3890f691d605b01c20932f58dd9511 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 28 Oct 2024 18:12:06 +0100 Subject: [PATCH 060/313] Bump Azurite to the latest version (#11411) --- .../java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java index 887758af4b28..7653fcce73ca 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java @@ -33,7 +33,7 @@ public class AzuriteContainer extends GenericContainer { private static final int DEFAULT_PORT = 10000; // default blob service port private static final String DEFAULT_IMAGE = "mcr.microsoft.com/azure-storage/azurite"; - private static final String DEFAULT_TAG = "3.30.0"; + private static final String DEFAULT_TAG = "3.33.0"; private static final String LOG_WAIT_REGEX = "Azurite Blob service is successfully listening at .*"; From a28ebf748cc00f66b80c797dc13181b4e81fe252 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 18:12:48 +0100 Subject: [PATCH 061/313] Build: Bump datamodel-code-generator from 0.26.1 to 0.26.2 (#11356) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.26.1 to 0.26.2. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.26.1...0.26.2) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index d15d7b224871..ba58048e02f7 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.1 -datamodel-code-generator==0.26.1 +datamodel-code-generator==0.26.2 From 6c58f5bb7f32ec3b322f68f15c82a62a95267e77 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Mon, 28 Oct 2024 13:32:19 -0400 Subject: [PATCH 062/313] Revert "Core: Snapshot `summary` map must have `operation` key (#11354)" (#11409) This reverts commit 7ad11b2df1a266d29f9e4f6bb5b499cb68c0afb7. --- .../org/apache/iceberg/SnapshotParser.java | 5 +- .../org/apache/iceberg/TestSnapshotJson.java | 77 ------------------- 2 files changed, 3 insertions(+), 79 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotParser.java b/core/src/main/java/org/apache/iceberg/SnapshotParser.java index 41b8e1499c0a..bc5ef6094695 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotParser.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotParser.java @@ -129,12 +129,13 @@ static Snapshot fromJson(JsonNode node) { "Cannot parse summary from non-object value: %s", sNode); - operation = JsonUtil.getString(OPERATION, sNode); ImmutableMap.Builder builder = ImmutableMap.builder(); Iterator fields = sNode.fieldNames(); while (fields.hasNext()) { String field = fields.next(); - if (!field.equals(OPERATION)) { + if (field.equals(OPERATION)) { + operation = JsonUtil.getString(OPERATION, sNode); + } else { builder.put(field, JsonUtil.getString(field, sNode)); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index 7fff5c5dddd9..ee1239074997 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -20,16 +20,11 @@ import static org.apache.iceberg.Files.localInput; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; import java.io.File; import java.io.IOException; import java.nio.file.Path; import java.util.List; -import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -40,56 +35,6 @@ public class TestSnapshotJson { public TableOperations ops = new LocalTableOperations(temp); - @Test - public void testToJsonWithoutOperation() throws IOException { - int snapshotId = 23; - Long parentId = null; - String manifestList = createManifestListWithManifestFiles(snapshotId, parentId); - - Snapshot expected = - new BaseSnapshot( - 0, snapshotId, parentId, System.currentTimeMillis(), null, null, 1, manifestList); - String json = SnapshotParser.toJson(expected); - - // Assert that summary field is not present in the JSON - assertThat(new ObjectMapper().readTree(json)).anyMatch(node -> !node.has("summary")); - } - - @Test - public void testToJsonWithOperation() throws IOException { - long parentId = 1; - long id = 2; - - String manifestList = createManifestListWithManifestFiles(id, parentId); - - Snapshot expected = - new BaseSnapshot( - 0, - id, - parentId, - System.currentTimeMillis(), - DataOperations.REPLACE, - ImmutableMap.of("files-added", "4", "files-deleted", "100"), - 3, - manifestList); - Map expectedSummary = - ImmutableMap.builder() - .putAll(expected.summary()) - .put("operation", expected.operation()) // operation should be part of the summary - .build(); - - String json = SnapshotParser.toJson(expected); - ObjectMapper objectMapper = new ObjectMapper(); - JsonNode jsonNode = objectMapper.readTree(json); - - assertThat(jsonNode.get("summary")).isNotNull(); - - Map actualSummary = - objectMapper.convertValue( - jsonNode.get("summary"), new TypeReference>() {}); - assertThat(actualSummary).isEqualTo(expectedSummary); - } - @Test public void testJsonConversion() throws IOException { int snapshotId = 23; @@ -214,28 +159,6 @@ public void testJsonConversionWithV1Manifests() { assertThat(snapshot.schemaId()).isEqualTo(expected.schemaId()); } - @Test - public void testJsonConversionSummaryWithoutOperationFails() { - String json = - String.format( - "{\n" - + " \"snapshot-id\" : 2,\n" - + " \"parent-snapshot-id\" : 1,\n" - + " \"timestamp-ms\" : %s,\n" - + " \"summary\" : {\n" - + " \"files-added\" : \"4\",\n" - + " \"files-deleted\" : \"100\"\n" - + " },\n" - + " \"manifests\" : [ \"/tmp/manifest1.avro\", \"/tmp/manifest2.avro\" ],\n" - + " \"schema-id\" : 3\n" - + "}", - System.currentTimeMillis()); - - assertThatThrownBy(() -> SnapshotParser.fromJson(json)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot parse missing string: operation"); - } - private String createManifestListWithManifestFiles(long snapshotId, Long parentSnapshotId) throws IOException { File manifestList = File.createTempFile("manifests", null, temp.toFile()); From e013c67f24f0c0c256dd6a0dfeb872b7959ac267 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Mon, 28 Oct 2024 18:42:08 +0100 Subject: [PATCH 063/313] Aliyun: Remove spring-boot dependency (#11291) --- .../apache/iceberg/aliyun/TestUtility.java | 2 +- .../aliyun/oss/TestOSSOutputStream.java | 2 +- .../aliyun/oss/mock/AliyunOSSMock.java | 569 ++++++++++++++++++ .../aliyun/oss/mock/AliyunOSSMockApp.java | 158 ----- .../oss/mock/AliyunOSSMockExtension.java | 23 +- .../mock/AliyunOSSMockLocalController.java | 522 ---------------- .../oss/mock/AliyunOSSMockLocalStore.java | 14 +- .../apache/iceberg/aliyun/oss/mock/Range.java | 43 -- build.gradle | 10 - gradle/libs.versions.toml | 5 - 10 files changed, 587 insertions(+), 761 deletions(-) create mode 100644 aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMock.java delete mode 100644 aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java delete mode 100644 aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java delete mode 100644 aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/Range.java diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java index 072886f6b86b..430eb6a50b42 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java @@ -65,7 +65,7 @@ public static AliyunOSSExtension initialize() { } else { LOG.info( "Initializing AliyunOSSExtension implementation with default AliyunOSSMockExtension"); - extension = AliyunOSSMockExtension.builder().silent().build(); + extension = AliyunOSSMockExtension.builder().build(); } return extension; diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java index 8fc661e5be10..9a7b774b28bd 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java @@ -70,7 +70,7 @@ public void testWrite() throws IOException { reset(ossMock); // Write large file. - writeAndVerify(ossMock, uri, randomData(32 * 1024 * 1024), arrayWrite); + writeAndVerify(ossMock, uri, randomData(32 * 1024), arrayWrite); verify(ossMock, times(1)).putObject(any()); reset(ossMock); } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMock.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMock.java new file mode 100644 index 000000000000..7894c1857d55 --- /dev/null +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMock.java @@ -0,0 +1,569 @@ +/* + * 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.aliyun.oss.mock; + +import com.aliyun.oss.OSSErrorCode; +import com.aliyun.oss.model.Bucket; +import com.sun.net.httpserver.HttpExchange; +import com.sun.net.httpserver.HttpHandler; +import com.sun.net.httpserver.HttpServer; +import java.io.FileInputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.Collections; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; + +public class AliyunOSSMock { + + static final String PROP_ROOT_DIR = "root-dir"; + static final String ROOT_DIR_DEFAULT = "/tmp"; + + static final String PROP_HTTP_PORT = "server.port"; + static final int PORT_HTTP_PORT_DEFAULT = 9393; + + private final AliyunOSSMockLocalStore localStore; + private final HttpServer httpServer; + + public static AliyunOSSMock start(Map properties) throws IOException { + AliyunOSSMock mock = + new AliyunOSSMock( + properties.getOrDefault(PROP_ROOT_DIR, ROOT_DIR_DEFAULT).toString(), + Integer.parseInt( + properties.getOrDefault(PROP_HTTP_PORT, PORT_HTTP_PORT_DEFAULT).toString())); + mock.start(); + return mock; + } + + private AliyunOSSMock(String rootDir, int serverPort) throws IOException { + localStore = new AliyunOSSMockLocalStore(rootDir); + httpServer = HttpServer.create(new InetSocketAddress("localhost", serverPort), 0); + } + + private void start() { + httpServer.createContext("/", new AliyunHttpHandler()); + httpServer.start(); + } + + public void stop() { + httpServer.stop(0); + } + + private class AliyunHttpHandler implements HttpHandler { + + @Override + public void handle(HttpExchange httpExchange) throws IOException { + String request = httpExchange.getRequestURI().getPath().substring(1); + String[] requests = request.split("/"); + String bucketName = requests[0]; + if (requests.length == 1) { + // bucket operations + if (httpExchange.getRequestMethod().equals("PUT")) { + putBucket(bucketName, httpExchange); + } + if (httpExchange.getRequestMethod().equals("DELETE")) { + deleteBucket(bucketName, httpExchange); + } + } else { + // object operations + String objectName = requests[1]; + if (objectName.contains("?")) { + objectName = objectName.substring(0, objectName.indexOf("?")); + } + if (httpExchange.getRequestMethod().equals("PUT")) { + putObject(bucketName, objectName, httpExchange); + } + if (httpExchange.getRequestMethod().equals("DELETE")) { + deleteObject(bucketName, objectName, httpExchange); + } + if (httpExchange.getRequestMethod().equals("HEAD")) { + getObjectMeta(bucketName, objectName, httpExchange); + } + if (httpExchange.getRequestMethod().equals("GET")) { + getObject(bucketName, objectName, httpExchange); + } + } + } + + private void putBucket(String bucketName, HttpExchange httpExchange) throws IOException { + if (localStore.getBucket(bucketName) != null) { + String errorMessage = + createErrorResponse( + OSSErrorCode.BUCKET_ALREADY_EXISTS, bucketName + " already exists."); + handleResponse(httpExchange, 409, errorMessage, "application/xml"); + return; + } + localStore.createBucket(bucketName); + handleResponse(httpExchange, 200, "OK", "application/xml"); + } + + private void deleteBucket(String bucketName, HttpExchange httpExchange) throws IOException { + verifyBucketExistence(bucketName, httpExchange); + try { + localStore.deleteBucket(bucketName); + } catch (Exception e) { + String errorMessage = + createErrorResponse( + OSSErrorCode.BUCKET_NOT_EMPTY, "The bucket you tried to delete is not empty."); + handleResponse(httpExchange, 409, errorMessage, "application/xml"); + } + handleResponse(httpExchange, 200, "OK", "application/xml"); + } + + private void putObject(String bucketName, String objectName, HttpExchange httpExchange) + throws IOException { + verifyBucketExistence(bucketName, httpExchange); + + try (InputStream inputStream = httpExchange.getRequestBody()) { + ObjectMetadata metadata = + localStore.putObject( + bucketName, + objectName, + inputStream, + httpExchange.getRequestHeaders().getFirst("Content-Type"), + httpExchange.getRequestHeaders().getFirst("Content-Headers"), + ImmutableMap.of()); + + httpExchange.getResponseHeaders().add("ETag", metadata.getContentMD5()); + httpExchange + .getResponseHeaders() + .add("Last-Modified", createDate(metadata.getLastModificationDate())); + handleResponse(httpExchange, 200, "OK", "text/plain"); + } catch (Exception e) { + handleResponse(httpExchange, 500, "Internal Server Error", "text/plain"); + } + } + + private void deleteObject(String bucketName, String objectName, HttpExchange httpExchange) + throws IOException { + verifyBucketExistence(bucketName, httpExchange); + localStore.deleteObject(bucketName, objectName); + + handleResponse(httpExchange, 200, "OK", "text/plain"); + } + + private void getObjectMeta(String bucketName, String objectName, HttpExchange httpExchange) + throws IOException { + verifyBucketExistence(bucketName, httpExchange); + ObjectMetadata metadata = verifyObjectExistence(bucketName, objectName); + + if (metadata == null) { + String errorMessage = + createErrorResponse(OSSErrorCode.NO_SUCH_KEY, "The specify oss key does not exists."); + handleResponse(httpExchange, 404, errorMessage, "application/xml"); + } else { + httpExchange.getResponseHeaders().add("ETag", metadata.getContentMD5()); + httpExchange + .getResponseHeaders() + .add("Last-Modified", createDate(metadata.getLastModificationDate())); + httpExchange + .getResponseHeaders() + .add("Content-Length", Long.toString(metadata.getContentLength())); + + handleResponse(httpExchange, 200, "OK", "text/plain"); + } + } + + private void getObject(String bucketName, String objectName, HttpExchange httpExchange) + throws IOException { + verifyBucketExistence(bucketName, httpExchange); + + String filename = objectName; + ObjectMetadata metadata = verifyObjectExistence(bucketName, filename); + + if (metadata == null) { + String errorMessage = + createErrorResponse(OSSErrorCode.NO_SUCH_KEY, "The specify oss key does not exists."); + handleResponse(httpExchange, 404, errorMessage, "application/xml"); + return; + } + + Object range = httpExchange.getRequestHeaders().get("Range"); + if (range != null) { + range = range.toString().replace("[bytes=", "").replace("]", ""); + String[] ranges = range.toString().split("-"); + long rangeStart = -1; + if (!ranges[0].isEmpty()) { + rangeStart = Long.parseLong(ranges[0]); + } + long rangeEnd = -1; + if (ranges.length == 2 && !ranges[1].isEmpty()) { + rangeEnd = Long.parseLong(ranges[1]); + } + if (rangeEnd == -1) { + rangeEnd = Long.MAX_VALUE; + if (rangeStart == -1) { + rangeStart = 0; + } + } + + long fileSize = metadata.getContentLength(); + long bytesToRead = Math.min(fileSize - 1, rangeEnd) - rangeStart + 1; + long skipSize = rangeStart; + if (rangeStart == -1) { + bytesToRead = Math.min(fileSize - 1, rangeEnd); + skipSize = fileSize - rangeEnd; + } + if (rangeEnd == -1) { + bytesToRead = fileSize - rangeStart; + } + if (bytesToRead < 0 || fileSize < rangeStart) { + httpExchange.sendResponseHeaders(416, 1); + return; + } + + httpExchange.getResponseHeaders().add("Accept-Ranges", "bytes"); + httpExchange + .getResponseHeaders() + .add( + "Content-Range", + "bytes " + + rangeStart + + "-" + + (bytesToRead + rangeStart + 1) + + "/" + + metadata.getContentLength()); + httpExchange.getResponseHeaders().add("ETag", metadata.getContentMD5()); + httpExchange + .getResponseHeaders() + .add("Last-Modified", createDate(metadata.getLastModificationDate())); + httpExchange.getResponseHeaders().add("Content-Type", metadata.getContentType()); + httpExchange.getResponseHeaders().add("Content-Length", Long.toString(bytesToRead)); + httpExchange.sendResponseHeaders(206, bytesToRead); + try (OutputStream outputStream = httpExchange.getResponseBody()) { + try (FileInputStream fis = new FileInputStream(metadata.getDataFile())) { + fis.skip(skipSize); + ByteStreams.copy(new BoundedInputStream(fis, bytesToRead), outputStream); + } + } + } else { + httpExchange.getResponseHeaders().add("Accept-Ranges", "bytes"); + httpExchange.getResponseHeaders().add("ETag", metadata.getContentMD5()); + httpExchange + .getResponseHeaders() + .add("Last-Modified", createDate(metadata.getLastModificationDate())); + httpExchange.getResponseHeaders().add("Content-Type", metadata.getContentType()); + httpExchange.sendResponseHeaders(200, metadata.getContentLength()); + + try (OutputStream outputStream = httpExchange.getResponseBody()) { + try (FileInputStream fis = new FileInputStream(metadata.getDataFile())) { + ByteStreams.copy(fis, outputStream); + } + } + } + } + + private void verifyBucketExistence(String bucketName, HttpExchange httpExchange) + throws IOException { + Bucket bucket = localStore.getBucket(bucketName); + if (bucket == null) { + String errorMessage = + createErrorResponse( + OSSErrorCode.NO_SUCH_BUCKET, "The specified bucket does not exist."); + handleResponse(httpExchange, 404, errorMessage, "application/xml"); + } + } + + private ObjectMetadata verifyObjectExistence(String bucketName, String fileName) { + ObjectMetadata objectMetadata = null; + try { + objectMetadata = localStore.getObjectMetadata(bucketName, fileName); + } catch (IOException e) { + // no-op + } + + return objectMetadata; + } + + private void handleResponse( + HttpExchange httpExchange, int responseCode, String responsePayload, String contentType) + throws IOException { + OutputStream outputStream = httpExchange.getResponseBody(); + httpExchange.getResponseHeaders().put("Content-Type", Collections.singletonList(contentType)); + httpExchange.sendResponseHeaders(responseCode, responsePayload.length()); + outputStream.write(responsePayload.getBytes()); + outputStream.flush(); + outputStream.close(); + } + + private String createErrorResponse(String errorCode, String message) { + StringBuilder builder = new StringBuilder(); + builder.append(""); + builder.append("").append(errorCode).append(""); + builder.append("").append(message).append(""); + builder.append(""); + return builder.toString(); + } + + private String createDate(long timestamp) { + java.util.Date date = new java.util.Date(timestamp); + ZonedDateTime dateTime = date.toInstant().atZone(ZoneId.of("GMT")); + return dateTime.format(DateTimeFormatter.RFC_1123_DATE_TIME); + } + } + + /** + * Reads bytes up to a maximum length, if its count goes above that, it stops. + * + *

This is useful to wrap ServletInputStreams. The ServletInputStream will block if you try to + * read content from it that isn't there, because it doesn't know whether the content hasn't + * arrived yet or whether the content has finished. So, one of these, initialized with the + * Content-length sent in the ServletInputStream's header, will stop it blocking, providing it's + * been sent with a correct content length. + * + *

This code is borrowed from `org.apache.commons:commons-io` + */ + public class BoundedInputStream extends FilterInputStream { + + /** The max count of bytes to read. */ + private final long maxCount; + + /** The count of bytes read. */ + private long count; + + /** The marked position. */ + private long mark = -1; + + /** Flag if close should be propagated. */ + private boolean propagateClose = true; + + /** + * Constructs a new {@link BoundedInputStream} that wraps the given input stream and is + * unlimited. + * + * @param in The wrapped input stream. + */ + public BoundedInputStream(final InputStream in) { + this(in, -1); + } + + /** + * Constructs a new {@link BoundedInputStream} that wraps the given input stream and limits it + * to a certain size. + * + * @param inputStream The wrapped input stream. + * @param maxLength The maximum number of bytes to return. + */ + public BoundedInputStream(final InputStream inputStream, final long maxLength) { + // Some badly designed methods - e.g. the servlet API - overload length + // such that "-1" means stream finished + super(inputStream); + this.maxCount = maxLength; + } + + /** {@inheritDoc} */ + @Override + public int available() throws IOException { + if (isMaxLength()) { + onMaxLength(maxCount, count); + return 0; + } + return in.available(); + } + + /** + * Invokes the delegate's {@code close()} method if {@link #isPropagateClose()} is {@code true}. + * + * @throws IOException if an I/O error occurs. + */ + @Override + public void close() throws IOException { + if (propagateClose) { + in.close(); + } + } + + /** + * Gets the count of bytes read. + * + * @return The count of bytes read. + * @since 2.12.0 + */ + public long getCount() { + return count; + } + + /** + * Gets the max count of bytes to read. + * + * @return The max count of bytes to read. + * @since 2.12.0 + */ + public long getMaxLength() { + return maxCount; + } + + private boolean isMaxLength() { + return maxCount >= 0 && count >= maxCount; + } + + /** + * Tests whether the {@link #close()} method should propagate to the underling {@link + * InputStream}. + * + * @return {@code true} if calling {@link #close()} propagates to the {@code close()} method of + * the underlying stream or {@code false} if it does not. + */ + public boolean isPropagateClose() { + return propagateClose; + } + + /** + * Sets whether the {@link #close()} method should propagate to the underling {@link + * InputStream}. + * + * @param propagateClose {@code true} if calling {@link #close()} propagates to the {@code + * close()} method of the underlying stream or {@code false} if it does not. + */ + public void setPropagateClose(final boolean propagateClose) { + this.propagateClose = propagateClose; + } + + /** + * Invokes the delegate's {@code mark(int)} method. + * + * @param readlimit read ahead limit + */ + @Override + public synchronized void mark(final int readlimit) { + in.mark(readlimit); + mark = count; + } + + /** + * Invokes the delegate's {@code markSupported()} method. + * + * @return true if mark is supported, otherwise false + */ + @Override + public boolean markSupported() { + return in.markSupported(); + } + + /** + * A caller has caused a request that would cross the {@code maxLength} boundary. + * + * @param maxLength The max count of bytes to read. + * @param bytesRead The count of bytes read. + * @throws IOException Subclasses may throw. + * @since 2.12.0 + */ + protected void onMaxLength(final long maxLength, final long bytesRead) throws IOException { + // for subclasses + } + + /** + * Invokes the delegate's {@code read()} method if the current position is less than the limit. + * + * @return the byte read or -1 if the end of stream or the limit has been reached. + * @throws IOException if an I/O error occurs. + */ + @Override + public int read() throws IOException { + if (isMaxLength()) { + onMaxLength(maxCount, count); + return -1; + } + final int result = in.read(); + count++; + return result; + } + + /** + * Invokes the delegate's {@code read(byte[])} method. + * + * @param b the buffer to read the bytes into + * @return the number of bytes read or -1 if the end of stream or the limit has been reached. + * @throws IOException if an I/O error occurs. + */ + @Override + public int read(final byte[] b) throws IOException { + return this.read(b, 0, b.length); + } + + /** + * Invokes the delegate's {@code read(byte[], int, int)} method. + * + * @param b the buffer to read the bytes into + * @param off The start offset + * @param len The number of bytes to read + * @return the number of bytes read or -1 if the end of stream or the limit has been reached. + * @throws IOException if an I/O error occurs. + */ + @Override + public int read(final byte[] b, final int off, final int len) throws IOException { + if (isMaxLength()) { + onMaxLength(maxCount, count); + return -1; + } + final long maxRead = maxCount >= 0 ? Math.min(len, maxCount - count) : len; + final int bytesRead = in.read(b, off, (int) maxRead); + + if (bytesRead == -1) { + return -1; + } + + count += bytesRead; + return bytesRead; + } + + /** + * Invokes the delegate's {@code reset()} method. + * + * @throws IOException if an I/O error occurs. + */ + @Override + public synchronized void reset() throws IOException { + in.reset(); + count = mark; + } + + /** + * Invokes the delegate's {@code skip(long)} method. + * + * @param n the number of bytes to skip + * @return the actual number of bytes skipped + * @throws IOException if an I/O error occurs. + */ + @Override + public long skip(final long n) throws IOException { + final long toSkip = maxCount >= 0 ? Math.min(n, maxCount - count) : n; + final long skippedBytes = in.skip(toSkip); + count += skippedBytes; + return skippedBytes; + } + + /** + * Invokes the delegate's {@code toString()} method. + * + * @return the delegate's {@code toString()} + */ + @Override + public String toString() { + return in.toString(); + } + } +} diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java deleted file mode 100644 index ea0ef0fe4de3..000000000000 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockApp.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.aliyun.oss.mock; - -import java.util.List; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.boot.Banner; -import org.springframework.boot.SpringApplication; -import org.springframework.boot.autoconfigure.EnableAutoConfiguration; -import org.springframework.boot.autoconfigure.security.servlet.SecurityAutoConfiguration; -import org.springframework.boot.builder.SpringApplicationBuilder; -import org.springframework.context.ConfigurableApplicationContext; -import org.springframework.context.annotation.Bean; -import org.springframework.context.annotation.ComponentScan; -import org.springframework.context.annotation.Configuration; -import org.springframework.core.convert.converter.Converter; -import org.springframework.http.MediaType; -import org.springframework.http.converter.xml.MappingJackson2XmlHttpMessageConverter; -import org.springframework.util.StringUtils; -import org.springframework.web.servlet.config.annotation.WebMvcConfigurer; - -@SuppressWarnings("checkstyle:AnnotationUseStyle") -@Configuration -@EnableAutoConfiguration( - exclude = {SecurityAutoConfiguration.class}, - excludeName = { - "org.springframework.boot.actuate.autoconfigure.security.servlet.ManagementWebSecurityAutoConfiguration" - }) -@ComponentScan -public class AliyunOSSMockApp { - - static final String PROP_ROOT_DIR = "root-dir"; - - static final String PROP_HTTP_PORT = "server.port"; - static final int PORT_HTTP_PORT_DEFAULT = 9393; - - static final String PROP_SILENT = "silent"; - - @Autowired private ConfigurableApplicationContext context; - - public static AliyunOSSMockApp start(Map properties, String... args) { - Map defaults = Maps.newHashMap(); - defaults.put(PROP_HTTP_PORT, PORT_HTTP_PORT_DEFAULT); - - Banner.Mode bannerMode = Banner.Mode.CONSOLE; - - if (Boolean.parseBoolean(String.valueOf(properties.remove(PROP_SILENT)))) { - defaults.put("logging.level.root", "WARN"); - bannerMode = Banner.Mode.OFF; - } - - final ConfigurableApplicationContext ctx = - new SpringApplicationBuilder(AliyunOSSMockApp.class) - .properties(defaults) - .properties(properties) - .bannerMode(bannerMode) - .run(args); - - return ctx.getBean(AliyunOSSMockApp.class); - } - - public void stop() { - SpringApplication.exit(context, () -> 0); - } - - @Configuration - static class Config implements WebMvcConfigurer { - - @Bean - Converter rangeConverter() { - return new RangeConverter(); - } - - /** - * Creates an HttpMessageConverter for XML. - * - * @return The configured {@link MappingJackson2XmlHttpMessageConverter}. - */ - @Bean - public MappingJackson2XmlHttpMessageConverter getMessageConverter() { - List mediaTypes = Lists.newArrayList(); - mediaTypes.add(MediaType.APPLICATION_XML); - mediaTypes.add(MediaType.APPLICATION_FORM_URLENCODED); - mediaTypes.add(MediaType.APPLICATION_OCTET_STREAM); - - final MappingJackson2XmlHttpMessageConverter xmlConverter = - new MappingJackson2XmlHttpMessageConverter(); - xmlConverter.setSupportedMediaTypes(mediaTypes); - - return xmlConverter; - } - } - - private static class RangeConverter implements Converter { - - private static final Pattern REQUESTED_RANGE_PATTERN = - Pattern.compile("^bytes=((\\d*)-(\\d*))((,\\d*-\\d*)*)"); - - @Override - public Range convert(String rangeString) { - Preconditions.checkNotNull(rangeString, "Range value should not be null."); - - final Range range; - - // parsing a range specification of format: "bytes=start-end" - multiple ranges not supported - final Matcher matcher = REQUESTED_RANGE_PATTERN.matcher(rangeString.trim()); - if (matcher.matches()) { - final String rangeStart = matcher.group(2); - final String rangeEnd = matcher.group(3); - - long start = StringUtils.isEmpty(rangeStart) ? -1L : Long.parseLong(rangeStart); - long end = StringUtils.isEmpty(rangeEnd) ? Long.MAX_VALUE : Long.parseLong(rangeEnd); - range = new Range(start, end); - - if (matcher.groupCount() == 5 && !"".equals(matcher.group(4))) { - throw new IllegalArgumentException( - "Unsupported range specification. Only single range specifications allowed"); - } - if (range.start() != -1 && range.start() < 0) { - throw new IllegalArgumentException( - "Unsupported range specification. A start byte must be supplied"); - } - - if (range.end() != -1 && range.end() < range.start()) { - throw new IllegalArgumentException( - "Range header is malformed. End byte is smaller than start byte."); - } - } else { - // Per Aliyun OSS behavior, return whole object content for illegal header - range = new Range(0, Long.MAX_VALUE); - } - - return range; - } - } -} diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java index 9aae5b777a8d..d4cb10615094 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java @@ -34,7 +34,7 @@ public class AliyunOSSMockExtension implements AliyunOSSExtension { private final Map properties; - private AliyunOSSMockApp ossMockApp; + private AliyunOSSMock ossMock; private AliyunOSSMockExtension(Map properties) { this.properties = properties; @@ -51,12 +51,16 @@ public String keyPrefix() { @Override public void start() { - ossMockApp = AliyunOSSMockApp.start(properties); + try { + ossMock = AliyunOSSMock.start(properties); + } catch (Exception e) { + throw new RuntimeException("Can't start OSS Mock"); + } } @Override public void stop() { - ossMockApp.stop(); + ossMock.stop(); } @Override @@ -65,12 +69,12 @@ public OSS createOSSClient() { String.format( "http://localhost:%s", properties.getOrDefault( - AliyunOSSMockApp.PROP_HTTP_PORT, AliyunOSSMockApp.PORT_HTTP_PORT_DEFAULT)); + AliyunOSSMock.PROP_HTTP_PORT, AliyunOSSMock.PORT_HTTP_PORT_DEFAULT)); return new OSSClientBuilder().build(endpoint, "foo", "bar"); } private File rootDir() { - Object rootDir = properties.get(AliyunOSSMockApp.PROP_ROOT_DIR); + Object rootDir = properties.get(AliyunOSSMock.PROP_ROOT_DIR); Preconditions.checkNotNull(rootDir, "Root directory cannot be null"); return new File(rootDir.toString()); } @@ -103,20 +107,15 @@ public void tearDownBucket(String bucket) { public static class Builder { private final Map props = Maps.newHashMap(); - public Builder silent() { - props.put(AliyunOSSMockApp.PROP_SILENT, true); - return this; - } - public AliyunOSSExtension build() { - String rootDir = (String) props.get(AliyunOSSMockApp.PROP_ROOT_DIR); + String rootDir = (String) props.get(AliyunOSSMock.PROP_ROOT_DIR); if (Strings.isNullOrEmpty(rootDir)) { File dir = new File( System.getProperty("java.io.tmpdir"), "oss-mock-file-store-" + System.currentTimeMillis()); rootDir = dir.getAbsolutePath(); - props.put(AliyunOSSMockApp.PROP_ROOT_DIR, rootDir); + props.put(AliyunOSSMock.PROP_ROOT_DIR, rootDir); } File root = new File(rootDir); root.deleteOnExit(); diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java deleted file mode 100644 index 7f7546ec233b..000000000000 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalController.java +++ /dev/null @@ -1,522 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.aliyun.oss.mock; - -import static org.springframework.http.HttpStatus.INTERNAL_SERVER_ERROR; -import static org.springframework.http.HttpStatus.OK; -import static org.springframework.http.HttpStatus.PARTIAL_CONTENT; -import static org.springframework.http.HttpStatus.REQUESTED_RANGE_NOT_SATISFIABLE; - -import com.aliyun.oss.OSSErrorCode; -import com.aliyun.oss.model.Bucket; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonRootName; -import java.io.FileInputStream; -import java.io.FilterInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import javax.servlet.ServletInputStream; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.http.HttpHeaders; -import org.springframework.http.MediaType; -import org.springframework.http.ResponseEntity; -import org.springframework.web.bind.annotation.ControllerAdvice; -import org.springframework.web.bind.annotation.ExceptionHandler; -import org.springframework.web.bind.annotation.PathVariable; -import org.springframework.web.bind.annotation.RequestHeader; -import org.springframework.web.bind.annotation.RequestMapping; -import org.springframework.web.bind.annotation.RequestMethod; -import org.springframework.web.bind.annotation.RestController; -import org.springframework.web.servlet.mvc.method.annotation.ResponseEntityExceptionHandler; - -@RestController -public class AliyunOSSMockLocalController { - private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockLocalController.class); - - @Autowired private AliyunOSSMockLocalStore localStore; - - private static String filenameFrom(@PathVariable String bucketName, HttpServletRequest request) { - String requestUri = request.getRequestURI(); - return requestUri.substring(requestUri.indexOf(bucketName) + bucketName.length() + 1); - } - - @RequestMapping(value = "/{bucketName}", method = RequestMethod.PUT, produces = "application/xml") - public void putBucket(@PathVariable String bucketName) throws IOException { - if (localStore.getBucket(bucketName) != null) { - throw new OssException( - 409, OSSErrorCode.BUCKET_ALREADY_EXISTS, bucketName + " already exists."); - } - - localStore.createBucket(bucketName); - } - - @RequestMapping( - value = "/{bucketName}", - method = RequestMethod.DELETE, - produces = "application/xml") - public void deleteBucket(@PathVariable String bucketName) throws IOException { - verifyBucketExistence(bucketName); - - localStore.deleteBucket(bucketName); - } - - @RequestMapping(value = "/{bucketName:.+}/**", method = RequestMethod.PUT) - public ResponseEntity putObject( - @PathVariable String bucketName, HttpServletRequest request) { - verifyBucketExistence(bucketName); - String filename = filenameFrom(bucketName, request); - try (ServletInputStream inputStream = request.getInputStream()) { - ObjectMetadata metadata = - localStore.putObject( - bucketName, - filename, - inputStream, - request.getContentType(), - request.getHeader(HttpHeaders.CONTENT_ENCODING), - ImmutableMap.of()); - - HttpHeaders responseHeaders = new HttpHeaders(); - responseHeaders.setETag("\"" + metadata.getContentMD5() + "\""); - responseHeaders.setLastModified(metadata.getLastModificationDate()); - - return new ResponseEntity<>(responseHeaders, OK); - } catch (Exception e) { - LOG.error("Failed to put object - bucket: {} - object: {}", bucketName, filename, e); - return new ResponseEntity<>(e.getMessage(), INTERNAL_SERVER_ERROR); - } - } - - @RequestMapping(value = "/{bucketName:.+}/**", method = RequestMethod.DELETE) - public void deleteObject(@PathVariable String bucketName, HttpServletRequest request) { - verifyBucketExistence(bucketName); - - localStore.deleteObject(bucketName, filenameFrom(bucketName, request)); - } - - @RequestMapping(value = "/{bucketName:.+}/**", method = RequestMethod.HEAD) - public ResponseEntity getObjectMeta( - @PathVariable String bucketName, HttpServletRequest request) { - verifyBucketExistence(bucketName); - ObjectMetadata metadata = verifyObjectExistence(bucketName, filenameFrom(bucketName, request)); - - HttpHeaders headers = new HttpHeaders(); - headers.setETag("\"" + metadata.getContentMD5() + "\""); - headers.setLastModified(metadata.getLastModificationDate()); - headers.setContentLength(metadata.getContentLength()); - - return new ResponseEntity<>(headers, OK); - } - - @SuppressWarnings("checkstyle:AnnotationUseStyle") - @RequestMapping( - value = "/{bucketName:.+}/**", - method = RequestMethod.GET, - produces = "application/xml") - public void getObject( - @PathVariable String bucketName, - @RequestHeader(value = "Range", required = false) Range range, - HttpServletRequest request, - HttpServletResponse response) - throws IOException { - verifyBucketExistence(bucketName); - - String filename = filenameFrom(bucketName, request); - ObjectMetadata metadata = verifyObjectExistence(bucketName, filename); - - if (range != null) { - long fileSize = metadata.getContentLength(); - long bytesToRead = Math.min(fileSize - 1, range.end()) - range.start() + 1; - long skipSize = range.start(); - if (range.start() == -1) { - bytesToRead = Math.min(fileSize - 1, range.end()); - skipSize = fileSize - range.end(); - } - if (range.end() == -1) { - bytesToRead = fileSize - range.start(); - } - if (bytesToRead < 0 || fileSize < range.start()) { - response.setStatus(REQUESTED_RANGE_NOT_SATISFIABLE.value()); - response.flushBuffer(); - return; - } - - response.setStatus(PARTIAL_CONTENT.value()); - response.setHeader(HttpHeaders.ACCEPT_RANGES, "bytes"); - response.setHeader( - HttpHeaders.CONTENT_RANGE, - String.format( - "bytes %s-%s/%s", - range.start(), bytesToRead + range.start() + 1, metadata.getContentLength())); - response.setHeader(HttpHeaders.ETAG, "\"" + metadata.getContentMD5() + "\""); - response.setDateHeader(HttpHeaders.LAST_MODIFIED, metadata.getLastModificationDate()); - response.setContentType(metadata.getContentType()); - response.setContentLengthLong(bytesToRead); - - try (OutputStream outputStream = response.getOutputStream()) { - try (FileInputStream fis = new FileInputStream(metadata.getDataFile())) { - fis.skip(skipSize); - ByteStreams.copy(new BoundedInputStream(fis, bytesToRead), outputStream); - } - } - } else { - response.setHeader(HttpHeaders.ACCEPT_RANGES, "bytes"); - response.setHeader(HttpHeaders.ETAG, "\"" + metadata.getContentMD5() + "\""); - response.setDateHeader(HttpHeaders.LAST_MODIFIED, metadata.getLastModificationDate()); - response.setContentType(metadata.getContentType()); - response.setContentLengthLong(metadata.getContentLength()); - - try (OutputStream outputStream = response.getOutputStream()) { - try (FileInputStream fis = new FileInputStream(metadata.getDataFile())) { - ByteStreams.copy(fis, outputStream); - } - } - } - } - - private void verifyBucketExistence(String bucketName) { - Bucket bucket = localStore.getBucket(bucketName); - if (bucket == null) { - throw new OssException( - 404, OSSErrorCode.NO_SUCH_BUCKET, "The specified bucket does not exist. "); - } - } - - private ObjectMetadata verifyObjectExistence(String bucketName, String filename) { - ObjectMetadata objectMetadata = null; - try { - objectMetadata = localStore.getObjectMetadata(bucketName, filename); - } catch (IOException e) { - LOG.error( - "Failed to get the object metadata, bucket: {}, object: {}.", bucketName, filename, e); - } - - if (objectMetadata == null) { - throw new OssException(404, OSSErrorCode.NO_SUCH_KEY, "The specify oss key does not exists."); - } - - return objectMetadata; - } - - @ControllerAdvice - public static class OSSMockExceptionHandler extends ResponseEntityExceptionHandler { - - @ExceptionHandler - public ResponseEntity handleOSSException(OssException ex) { - LOG.info("Responding with status {} - {}, {}", ex.status, ex.code, ex.message); - - ErrorResponse errorResponse = new ErrorResponse(); - errorResponse.setCode(ex.getCode()); - errorResponse.setMessage(ex.getMessage()); - - HttpHeaders headers = new HttpHeaders(); - headers.setContentType(MediaType.APPLICATION_XML); - - return ResponseEntity.status(ex.status).headers(headers).body(errorResponse); - } - } - - public static class OssException extends RuntimeException { - - private final int status; - private final String code; - private final String message; - - public OssException(final int status, final String code, final String message) { - super(message); - this.status = status; - this.code = code; - this.message = message; - } - - public String getCode() { - return code; - } - - @Override - public String getMessage() { - return message; - } - } - - @JsonRootName("Error") - public static class ErrorResponse { - @JsonProperty("Code") - private String code; - - @JsonProperty("Message") - private String message; - - public void setCode(String code) { - this.code = code; - } - - public void setMessage(String message) { - this.message = message; - } - } - - /** - * Reads bytes up to a maximum length, if its count goes above that, it stops. - * - *

This is useful to wrap ServletInputStreams. The ServletInputStream will block if you try to - * read content from it that isn't there, because it doesn't know whether the content hasn't - * arrived yet or whether the content has finished. So, one of these, initialized with the - * Content-length sent in the ServletInputStream's header, will stop it blocking, providing it's - * been sent with a correct content length. - * - *

This code is borrowed from `org.apache.commons:commons-io` - */ - public class BoundedInputStream extends FilterInputStream { - - /** The max count of bytes to read. */ - private final long maxCount; - - /** The count of bytes read. */ - private long count; - - /** The marked position. */ - private long mark = -1; - - /** Flag if close should be propagated. */ - private boolean propagateClose = true; - - /** - * Constructs a new {@link BoundedInputStream} that wraps the given input stream and is - * unlimited. - * - * @param in The wrapped input stream. - */ - public BoundedInputStream(final InputStream in) { - this(in, -1); - } - - /** - * Constructs a new {@link BoundedInputStream} that wraps the given input stream and limits it - * to a certain size. - * - * @param inputStream The wrapped input stream. - * @param maxLength The maximum number of bytes to return. - */ - public BoundedInputStream(final InputStream inputStream, final long maxLength) { - // Some badly designed methods - e.g. the servlet API - overload length - // such that "-1" means stream finished - super(inputStream); - this.maxCount = maxLength; - } - - /** {@inheritDoc} */ - @Override - public int available() throws IOException { - if (isMaxLength()) { - onMaxLength(maxCount, count); - return 0; - } - return in.available(); - } - - /** - * Invokes the delegate's {@code close()} method if {@link #isPropagateClose()} is {@code true}. - * - * @throws IOException if an I/O error occurs. - */ - @Override - public void close() throws IOException { - if (propagateClose) { - in.close(); - } - } - - /** - * Gets the count of bytes read. - * - * @return The count of bytes read. - * @since 2.12.0 - */ - public long getCount() { - return count; - } - - /** - * Gets the max count of bytes to read. - * - * @return The max count of bytes to read. - * @since 2.12.0 - */ - public long getMaxLength() { - return maxCount; - } - - private boolean isMaxLength() { - return maxCount >= 0 && count >= maxCount; - } - - /** - * Tests whether the {@link #close()} method should propagate to the underling {@link - * InputStream}. - * - * @return {@code true} if calling {@link #close()} propagates to the {@code close()} method of - * the underlying stream or {@code false} if it does not. - */ - public boolean isPropagateClose() { - return propagateClose; - } - - /** - * Sets whether the {@link #close()} method should propagate to the underling {@link - * InputStream}. - * - * @param propagateClose {@code true} if calling {@link #close()} propagates to the {@code - * close()} method of the underlying stream or {@code false} if it does not. - */ - public void setPropagateClose(final boolean propagateClose) { - this.propagateClose = propagateClose; - } - - /** - * Invokes the delegate's {@code mark(int)} method. - * - * @param readlimit read ahead limit - */ - @Override - public synchronized void mark(final int readlimit) { - in.mark(readlimit); - mark = count; - } - - /** - * Invokes the delegate's {@code markSupported()} method. - * - * @return true if mark is supported, otherwise false - */ - @Override - public boolean markSupported() { - return in.markSupported(); - } - - /** - * A caller has caused a request that would cross the {@code maxLength} boundary. - * - * @param maxLength The max count of bytes to read. - * @param bytesRead The count of bytes read. - * @throws IOException Subclasses may throw. - * @since 2.12.0 - */ - protected void onMaxLength(final long maxLength, final long bytesRead) throws IOException { - // for subclasses - } - - /** - * Invokes the delegate's {@code read()} method if the current position is less than the limit. - * - * @return the byte read or -1 if the end of stream or the limit has been reached. - * @throws IOException if an I/O error occurs. - */ - @Override - public int read() throws IOException { - if (isMaxLength()) { - onMaxLength(maxCount, count); - return -1; - } - final int result = in.read(); - count++; - return result; - } - - /** - * Invokes the delegate's {@code read(byte[])} method. - * - * @param b the buffer to read the bytes into - * @return the number of bytes read or -1 if the end of stream or the limit has been reached. - * @throws IOException if an I/O error occurs. - */ - @Override - public int read(final byte[] b) throws IOException { - return this.read(b, 0, b.length); - } - - /** - * Invokes the delegate's {@code read(byte[], int, int)} method. - * - * @param b the buffer to read the bytes into - * @param off The start offset - * @param len The number of bytes to read - * @return the number of bytes read or -1 if the end of stream or the limit has been reached. - * @throws IOException if an I/O error occurs. - */ - @Override - public int read(final byte[] b, final int off, final int len) throws IOException { - if (isMaxLength()) { - onMaxLength(maxCount, count); - return -1; - } - final long maxRead = maxCount >= 0 ? Math.min(len, maxCount - count) : len; - final int bytesRead = in.read(b, off, (int) maxRead); - - if (bytesRead == -1) { - return -1; - } - - count += bytesRead; - return bytesRead; - } - - /** - * Invokes the delegate's {@code reset()} method. - * - * @throws IOException if an I/O error occurs. - */ - @Override - public synchronized void reset() throws IOException { - in.reset(); - count = mark; - } - - /** - * Invokes the delegate's {@code skip(long)} method. - * - * @param n the number of bytes to skip - * @return the actual number of bytes skipped - * @throws IOException if an I/O error occurs. - */ - @Override - public long skip(final long n) throws IOException { - final long toSkip = maxCount >= 0 ? Math.min(n, maxCount - count) : n; - final long skippedBytes = in.skip(toSkip); - count += skippedBytes; - return skippedBytes; - } - - /** - * Invokes the delegate's {@code toString()} method. - * - * @return the delegate's {@code toString()} - */ - @Override - public String toString() { - return in.toString(); - } - } -} diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java index f7a4b72e4b97..521b87e31e80 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java @@ -46,11 +46,7 @@ import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Value; -import org.springframework.http.MediaType; -import org.springframework.stereotype.Component; -@Component public class AliyunOSSMockLocalStore { private static final Logger LOG = LoggerFactory.getLogger(AliyunOSSMockLocalStore.class); @@ -61,8 +57,7 @@ public class AliyunOSSMockLocalStore { private final ObjectMapper objectMapper = new ObjectMapper(); - public AliyunOSSMockLocalStore( - @Value("${" + AliyunOSSMockApp.PROP_ROOT_DIR + ":}") String rootDir) { + public AliyunOSSMockLocalStore(String rootDir) { Preconditions.checkNotNull(rootDir, "Root directory cannot be null"); this.root = new File(rootDir); @@ -121,8 +116,7 @@ void deleteBucket(String bucketName) throws IOException { File dir = new File(root, bucket.getName()); if (Files.walk(dir.toPath()).anyMatch(p -> p.toFile().isFile())) { - throw new AliyunOSSMockLocalController.OssException( - 409, OSSErrorCode.BUCKET_NOT_EMPTY, "The bucket you tried to delete is not empty. "); + throw new RuntimeException(OSSErrorCode.BUCKET_NOT_EMPTY); } try (Stream walk = Files.walk(dir.toPath())) { @@ -156,7 +150,9 @@ ObjectMetadata putObject( metadata.setContentLength(dataFile.length()); metadata.setContentMD5(md5sum(dataFile.getAbsolutePath())); metadata.setContentType( - contentType != null ? contentType : MediaType.APPLICATION_OCTET_STREAM_VALUE); + contentType != null + ? contentType + : "application/octet"); // MediaType.APPLICATION_OCTET_STREAM_VALUE metadata.setContentEncoding(contentEncoding); metadata.setDataFile(dataFile.getAbsolutePath()); metadata.setMetaFile(metaFile.getAbsolutePath()); diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/Range.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/Range.java deleted file mode 100644 index ff66e5c2a1cb..000000000000 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/Range.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.aliyun.oss.mock; - -public class Range { - - private final long start; - private final long end; - - public Range(long start, long end) { - this.start = start; - this.end = end; - } - - public long start() { - return start; - } - - public long end() { - return end; - } - - @Override - public String toString() { - return String.format("%d-%d", start, end); - } -} diff --git a/build.gradle b/build.gradle index a654e1cba5fb..abab68ca4b8b 100644 --- a/build.gradle +++ b/build.gradle @@ -438,16 +438,6 @@ project(':iceberg-aliyun') { testImplementation platform(libs.jackson.bom) testImplementation "com.fasterxml.jackson.dataformat:jackson-dataformat-xml" testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation libs.spring.web - testImplementation(libs.spring.boot.starter.jetty) { - exclude module: 'logback-classic' - exclude group: 'org.eclipse.jetty.websocket', module: 'javax-websocket-server-impl' - exclude group: 'org.eclipse.jetty.websocket', module: 'websocket-server' - } - testImplementation(libs.spring.boot.starter.web) { - exclude module: 'logback-classic' - exclude module: 'spring-boot-starter-logging' - } } } diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 55aa97c350a9..c84341dea9e8 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -84,8 +84,6 @@ snowflake-jdbc = "3.19.1" spark-hive33 = "3.3.4" spark-hive34 = "3.4.4" spark-hive35 = "3.5.2" -spring-boot = "2.7.18" -spring-web = "5.3.39" sqlite-jdbc = "3.47.0.0" testcontainers = "1.20.3" tez010 = "0.10.4" @@ -219,9 +217,6 @@ nessie-jaxrs-testextension = { module = "org.projectnessie.nessie:nessie-jaxrs-t nessie-versioned-storage-inmemory-tests = { module = "org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests", version.ref = "nessie" } nessie-versioned-storage-testextension = { module = "org.projectnessie.nessie:nessie-versioned-storage-testextension", version.ref = "nessie" } orc-tools = { module = "org.apache.orc:orc-tools", version.ref = "orc" } -spring-boot-starter-jetty = { module = "org.springframework.boot:spring-boot-starter-jetty", version.ref = "spring-boot" } -spring-boot-starter-web = { module = "org.springframework.boot:spring-boot-starter-web", version.ref = "spring-boot" } -spring-web = { module = "org.springframework:spring-web", version.ref = "spring-web" } sqlite-jdbc = { module = "org.xerial:sqlite-jdbc", version.ref = "sqlite-jdbc" } testcontainers = { module = "org.testcontainers:testcontainers", version.ref = "testcontainers" } testcontainers-junit-jupiter = { module = "org.testcontainers:junit-jupiter", version.ref = "testcontainers" } From 7ac617a5a8b0dedbaaa6e19caedfd846968c7cac Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 20:55:22 +0100 Subject: [PATCH 064/313] Build: Bump com.azure:azure-sdk-bom from 1.2.25 to 1.2.28 (#11267) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.2.25 to 1.2.28. - [Release notes](https://github.com/azure/azure-sdk-for-java/releases) - [Commits](https://github.com/azure/azure-sdk-for-java/compare/azure-sdk-bom_1.2.25...azure-sdk-bom_1.2.28) --- updated-dependencies: - dependency-name: com.azure:azure-sdk-bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index c84341dea9e8..13ce4da5471b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -30,7 +30,7 @@ avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" awssdk-bom = "2.29.1" -azuresdk-bom = "1.2.25" +azuresdk-bom = "1.2.28" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" calcite = "1.10.0" From 47eac5221bd59ce0228f8e01825b35f8be7bf0c7 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 29 Oct 2024 03:57:39 +0800 Subject: [PATCH 065/313] Spark: Flaky test due temp directory (#10811) --- .../iceberg/spark/source/TestDataFrameWrites.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index bf49bfba550f..01c36b824ea6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -28,6 +28,7 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.nio.file.NoSuchFileException; import java.util.Arrays; import java.util.Collection; import java.util.Iterator; @@ -35,6 +36,7 @@ import java.util.Map; import java.util.Random; import org.apache.avro.generic.GenericData.Record; +import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.Files; import org.apache.iceberg.Parameter; @@ -419,5 +421,13 @@ public void testFaultToleranceOnWrite() throws IOException { assertThat(snapshotBeforeFailingWrite).isEqualTo(snapshotAfterFailingWrite); assertThat(resultBeforeFailingWrite).isEqualTo(resultAfterFailingWrite); + + while (location.exists()) { + try { + FileUtils.deleteDirectory(location); + } catch (NoSuchFileException e) { + // ignore NoSuchFileException when a file is already deleted + } + } } } From 1e3ee1e4e80873018af716a190e541925f09c285 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 28 Oct 2024 22:02:55 +0100 Subject: [PATCH 066/313] Core: Add portable Roaring bitmap for row positions (#11372) --- LICENSE | 1 + .../RoaringPositionBitmapBenchmark.java | 162 ++++++ .../deletes/RoaringPositionBitmap.java | 318 +++++++++++ .../deletes/TestRoaringPositionBitmap.java | 515 ++++++++++++++++++ .../apache/iceberg/deletes/64map32bitvals.bin | Bin 0 -> 48 bytes .../org/apache/iceberg/deletes/64mapempty.bin | Bin 0 -> 8 bytes .../apache/iceberg/deletes/64maphighvals.bin | Bin 0 -> 1086 bytes .../iceberg/deletes/64mapspreadvals.bin | Bin 0 -> 408 bytes 8 files changed, 996 insertions(+) create mode 100644 core/src/jmh/java/org/apache/iceberg/deletes/RoaringPositionBitmapBenchmark.java create mode 100644 core/src/main/java/org/apache/iceberg/deletes/RoaringPositionBitmap.java create mode 100644 core/src/test/java/org/apache/iceberg/deletes/TestRoaringPositionBitmap.java create mode 100644 core/src/test/resources/org/apache/iceberg/deletes/64map32bitvals.bin create mode 100644 core/src/test/resources/org/apache/iceberg/deletes/64mapempty.bin create mode 100644 core/src/test/resources/org/apache/iceberg/deletes/64maphighvals.bin create mode 100644 core/src/test/resources/org/apache/iceberg/deletes/64mapspreadvals.bin diff --git a/LICENSE b/LICENSE index efb46dab44da..76f6113d9811 100644 --- a/LICENSE +++ b/LICENSE @@ -298,6 +298,7 @@ License: https://www.apache.org/licenses/LICENSE-2.0 This product includes code from Delta Lake. * AssignmentAlignmentSupport is an independent development but UpdateExpressionsSupport in Delta was used as a reference. +* RoaringPositionBitmap is a Java implementation of RoaringBitmapArray in Delta. Copyright: 2020 The Delta Lake Project Authors. Home page: https://delta.io/ diff --git a/core/src/jmh/java/org/apache/iceberg/deletes/RoaringPositionBitmapBenchmark.java b/core/src/jmh/java/org/apache/iceberg/deletes/RoaringPositionBitmapBenchmark.java new file mode 100644 index 000000000000..1cbc39583fbc --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/deletes/RoaringPositionBitmapBenchmark.java @@ -0,0 +1,162 @@ +/* + * 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.deletes; + +import java.util.Random; +import java.util.concurrent.TimeUnit; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.roaringbitmap.longlong.Roaring64Bitmap; + +/** + * A benchmark that evaluates the performance of {@link RoaringPositionBitmap}. + * + *

To run this benchmark: + * ./gradlew :iceberg-core:jmh + * -PjmhIncludeRegex=RoaringPositionBitmapBenchmark + * -PjmhOutputPath=benchmark/roaring-position-bitmap-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 5, timeUnit = TimeUnit.MINUTES) +public class RoaringPositionBitmapBenchmark { + + private static final Random RANDOM = new Random(); + private static final int TOTAL_POSITIONS = 5_000_000; + private static final long STEP = 5L; + + private long[] orderedPositions; + private long[] shuffledPositions; + + @Setup + public void setupBenchmark() { + this.orderedPositions = generateOrderedPositions(); + this.shuffledPositions = generateShuffledPositions(); + } + + @Benchmark + @Threads(1) + public void addOrderedPositionsIcebergBitmap(Blackhole blackhole) { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + for (long position : orderedPositions) { + bitmap.set(position); + } + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addOrderedPositionsLibraryBitmap(Blackhole blackhole) { + Roaring64Bitmap bitmap = new Roaring64Bitmap(); + for (long position : orderedPositions) { + bitmap.add(position); + } + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addShuffledPositionsIcebergBitmap(Blackhole blackhole) { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + for (long position : shuffledPositions) { + bitmap.set(position); + } + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addShuffledPositionsLibraryBitmap(Blackhole blackhole) { + Roaring64Bitmap bitmap = new Roaring64Bitmap(); + for (long position : shuffledPositions) { + bitmap.add(position); + } + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addAndCheckPositionsIcebergBitmap(Blackhole blackhole) { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + for (long position : shuffledPositions) { + bitmap.set(position); + } + + for (long position = 0; position <= TOTAL_POSITIONS * STEP; position++) { + bitmap.contains(position); + } + + blackhole.consume(bitmap); + } + + @Benchmark + @Threads(1) + public void addAndCheckPositionsLibraryBitmap(Blackhole blackhole) { + Roaring64Bitmap bitmap = new Roaring64Bitmap(); + + for (long position : shuffledPositions) { + bitmap.add(position); + } + + for (long position = 0; position <= TOTAL_POSITIONS * STEP; position++) { + bitmap.contains(position); + } + + blackhole.consume(bitmap); + } + + private static long[] generateOrderedPositions() { + long[] positions = new long[TOTAL_POSITIONS]; + for (int index = 0; index < TOTAL_POSITIONS; index++) { + positions[index] = index * STEP; + } + return positions; + } + + private static long[] generateShuffledPositions() { + long[] positions = generateOrderedPositions(); + shuffle(positions); + return positions; + } + + private static void shuffle(long[] array) { + for (int index = array.length - 1; index > 0; index--) { + // swap with an element at a random index between 0 and index + int thatIndex = RANDOM.nextInt(index + 1); + long temp = array[index]; + array[index] = array[thatIndex]; + array[thatIndex] = temp; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/deletes/RoaringPositionBitmap.java b/core/src/main/java/org/apache/iceberg/deletes/RoaringPositionBitmap.java new file mode 100644 index 000000000000..eec130743d85 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/deletes/RoaringPositionBitmap.java @@ -0,0 +1,318 @@ +/* + * 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.deletes; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import java.util.function.LongConsumer; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.roaringbitmap.RoaringBitmap; + +/** + * A bitmap that supports positive 64-bit positions (the most significant bit must be 0), but is + * optimized for cases where most positions fit in 32 bits by using an array of 32-bit Roaring + * bitmaps. The internal bitmap array is grown as needed to accommodate the largest position. + * + *

Incoming 64-bit positions are divided into a 32-bit "key" using the most significant 4 bytes + * and a 32-bit position using the least significant 4 bytes. For each key in the set of positions, + * a 32-bit Roaring bitmap is maintained to store a set of 32-bit positions for that key. + * + *

To test whether a certain position is set, its most significant 4 bytes (the key) are used to + * find a 32-bit bitmap and the least significant 4 bytes are tested for inclusion in the bitmap. If + * a bitmap is not found for the key, then the position is not set. + * + *

Positions must range from 0 (inclusive) to {@link #MAX_POSITION} (inclusive). This class + * cannot handle positions with the key equal to Integer.MAX_VALUE because the length of the + * internal bitmap array is a signed 32-bit integer, which must be greater than or equal to 0. + * Supporting Integer.MAX_VALUE as a key would require allocating a bitmap array with size + * Integer.MAX_VALUE + 1, triggering an integer overflow. + */ +class RoaringPositionBitmap { + + static final long MAX_POSITION = toPosition(Integer.MAX_VALUE - 1, Integer.MIN_VALUE); + private static final RoaringBitmap[] EMPTY_BITMAP_ARRAY = new RoaringBitmap[0]; + private static final long BITMAP_COUNT_SIZE_BYTES = 8L; + private static final long BITMAP_KEY_SIZE_BYTES = 4L; + + private RoaringBitmap[] bitmaps; + + RoaringPositionBitmap() { + this.bitmaps = EMPTY_BITMAP_ARRAY; + } + + private RoaringPositionBitmap(RoaringBitmap[] bitmaps) { + this.bitmaps = bitmaps; + } + + /** + * Sets a position in the bitmap. + * + * @param pos the position + */ + public void set(long pos) { + validatePosition(pos); + int key = key(pos); + int pos32Bits = pos32Bits(pos); + allocateBitmapsIfNeeded(key + 1 /* required bitmap array length */); + bitmaps[key].add(pos32Bits); + } + + /** + * Sets a range of positions in the bitmap. + * + * @param posStartInclusive the start position of the range (inclusive) + * @param posEndExclusive the end position of the range (exclusive) + */ + public void setRange(long posStartInclusive, long posEndExclusive) { + for (long pos = posStartInclusive; pos < posEndExclusive; pos++) { + set(pos); + } + } + + /** + * Sets all positions from the other bitmap in this bitmap, modifying this bitmap in place. + * + * @param that the other bitmap + */ + public void setAll(RoaringPositionBitmap that) { + allocateBitmapsIfNeeded(that.bitmaps.length); + for (int key = 0; key < that.bitmaps.length; key++) { + bitmaps[key].or(that.bitmaps[key]); + } + } + + /** + * Checks if a position is set in the bitmap. + * + * @param pos the position + * @return true if the position is set in this bitmap, false otherwise + */ + public boolean contains(long pos) { + validatePosition(pos); + int key = key(pos); + int pos32Bits = pos32Bits(pos); + return key < bitmaps.length && bitmaps[key].contains(pos32Bits); + } + + /** + * Indicates whether the bitmap has any positions set. + * + * @return true if the bitmap is empty, false otherwise + */ + public boolean isEmpty() { + return cardinality() == 0; + } + + /** + * Returns the number of set positions in the bitmap. + * + * @return the number of set positions + */ + public long cardinality() { + long cardinality = 0L; + for (RoaringBitmap bitmap : bitmaps) { + cardinality += bitmap.getLongCardinality(); + } + return cardinality; + } + + /** + * Applies run-length encoding wherever it is more space efficient. + * + * @return whether the bitmap was changed + */ + public boolean runLengthEncode() { + boolean changed = false; + for (RoaringBitmap bitmap : bitmaps) { + changed |= bitmap.runOptimize(); + } + return changed; + } + + /** + * Iterates over all positions in the bitmap. + * + * @param consumer a consumer for positions + */ + public void forEach(LongConsumer consumer) { + for (int key = 0; key < bitmaps.length; key++) { + forEach(key, bitmaps[key], consumer); + } + } + + @VisibleForTesting + int allocatedBitmapCount() { + return bitmaps.length; + } + + private void allocateBitmapsIfNeeded(int requiredLength) { + if (bitmaps.length < requiredLength) { + if (bitmaps.length == 0 && requiredLength == 1) { + this.bitmaps = new RoaringBitmap[] {new RoaringBitmap()}; + } else { + RoaringBitmap[] newBitmaps = new RoaringBitmap[requiredLength]; + System.arraycopy(bitmaps, 0, newBitmaps, 0, bitmaps.length); + for (int key = bitmaps.length; key < requiredLength; key++) { + newBitmaps[key] = new RoaringBitmap(); + } + this.bitmaps = newBitmaps; + } + } + } + + /** + * Returns the number of bytes required to serialize the bitmap. + * + * @return the serialized size in bytes + */ + public long serializedSizeInBytes() { + long size = BITMAP_COUNT_SIZE_BYTES; + for (RoaringBitmap bitmap : bitmaps) { + size += BITMAP_KEY_SIZE_BYTES + bitmap.serializedSizeInBytes(); + } + return size; + } + + /** + * Serializes the bitmap using the portable serialization format described below. + * + *

    + *
  • The number of 32-bit Roaring bitmaps, serialized as 8 bytes + *
  • For each 32-bit Roaring bitmap, ordered by unsigned comparison of the 32-bit keys: + *
      + *
    • The key stored as 4 bytes + *
    • Serialized 32-bit Roaring bitmap using the standard format + *
    + *
+ * + *

Note the byte order of the buffer must be little-endian. + * + * @param buffer the buffer to write to + * @see Roaring bitmap spec + */ + public void serialize(ByteBuffer buffer) { + validateByteOrder(buffer); + buffer.putLong(bitmaps.length); + for (int key = 0; key < bitmaps.length; key++) { + buffer.putInt(key); + bitmaps[key].serialize(buffer); + } + } + + /** + * Deserializes a bitmap from a buffer, assuming the portable serialization format. + * + * @param buffer the buffer to read from + * @return a new bitmap instance with the deserialized data + */ + public static RoaringPositionBitmap deserialize(ByteBuffer buffer) { + validateByteOrder(buffer); + + // the bitmap array may be sparse with more elements than the number of read bitmaps + int remainingBitmapCount = readBitmapCount(buffer); + List bitmaps = Lists.newArrayListWithExpectedSize(remainingBitmapCount); + int lastKey = -1; + + while (remainingBitmapCount > 0) { + int key = readKey(buffer, lastKey); + + // fill gaps as the bitmap array may be sparse + while (lastKey < key - 1) { + bitmaps.add(new RoaringBitmap()); + lastKey++; + } + + RoaringBitmap bitmap = readBitmap(buffer); + bitmaps.add(bitmap); + + lastKey = key; + remainingBitmapCount--; + } + + return new RoaringPositionBitmap(bitmaps.toArray(EMPTY_BITMAP_ARRAY)); + } + + private static void validateByteOrder(ByteBuffer buffer) { + Preconditions.checkArgument( + buffer.order() == ByteOrder.LITTLE_ENDIAN, + "Roaring bitmap serialization requires little-endian byte order"); + } + + private static int readBitmapCount(ByteBuffer buffer) { + long bitmapCount = buffer.getLong(); + Preconditions.checkArgument( + bitmapCount >= 0 && bitmapCount <= Integer.MAX_VALUE, + "Invalid bitmap count: %s", + bitmapCount); + return (int) bitmapCount; + } + + private static int readKey(ByteBuffer buffer, int lastKey) { + int key = buffer.getInt(); + Preconditions.checkArgument(key >= 0, "Invalid unsigned key: %s", key); + Preconditions.checkArgument(key <= Integer.MAX_VALUE - 1, "Key is too large: %s", key); + Preconditions.checkArgument(key > lastKey, "Keys must be sorted in ascending order"); + return key; + } + + private static RoaringBitmap readBitmap(ByteBuffer buffer) { + try { + RoaringBitmap bitmap = new RoaringBitmap(); + bitmap.deserialize(buffer); + buffer.position(buffer.position() + bitmap.serializedSizeInBytes()); + return bitmap; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + // extracts high 32 bits from a 64-bit position (i.e. key) + private static int key(long pos) { + return (int) (pos >> 32); + } + + // extracts low 32 bits from a 64-bit position (i.e. 32-bit position) + private static int pos32Bits(long pos) { + return (int) pos; + } + + // combines high and low 32 bits into a 64-bit position + // the low 32 bits must be bit-masked to avoid sign extension + private static long toPosition(int key, int pos32Bits) { + return (((long) key) << 32) | (((long) pos32Bits) & 0xFFFFFFFFL); + } + + // iterates over 64-bit positions, reconstructing them from keys and 32-bit positions + private static void forEach(int key, RoaringBitmap bitmap, LongConsumer consumer) { + bitmap.forEach((int pos32Bits) -> consumer.accept(toPosition(key, pos32Bits))); + } + + private static void validatePosition(long pos) { + Preconditions.checkArgument( + pos >= 0 && pos <= MAX_POSITION, + "Bitmap supports positions that are >= 0 and <= %s: %s", + MAX_POSITION, + pos); + } +} diff --git a/core/src/test/java/org/apache/iceberg/deletes/TestRoaringPositionBitmap.java b/core/src/test/java/org/apache/iceberg/deletes/TestRoaringPositionBitmap.java new file mode 100644 index 000000000000..2daf0382973b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/deletes/TestRoaringPositionBitmap.java @@ -0,0 +1,515 @@ +/* + * 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.deletes; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import java.util.Random; +import java.util.Set; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.io.Resources; +import org.apache.iceberg.util.Pair; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRoaringPositionBitmap { + + private static final long BITMAP_SIZE = 0xFFFFFFFFL; + private static final long BITMAP_OFFSET = BITMAP_SIZE + 1L; + private static final long CONTAINER_SIZE = Character.MAX_VALUE; + private static final long CONTAINER_OFFSET = CONTAINER_SIZE + 1L; + private static final int VALIDATION_LOOKUP_COUNT = 20_000; + private static final Set SUPPORTED_OFFICIAL_EXAMPLE_FILES = + ImmutableSet.of("64map32bitvals.bin", "64mapempty.bin", "64mapspreadvals.bin"); + + @Parameters(name = "seed = {0}, validationSeed = {1}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + Random random = new Random(); + long seed = random.nextLong(); + long validationSeed = random.nextLong(); + parameters.add(new Object[] {seed, validationSeed}); + return parameters; + } + + @Parameter(index = 0) + private long seed; + + @Parameter(index = 1) + private long validationSeed; + + @TestTemplate + public void testAdd() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + bitmap.set(10L); + assertThat(bitmap.contains(10L)).isTrue(); + + bitmap.set(0L); + assertThat(bitmap.contains(0L)).isTrue(); + + bitmap.set(10L); + assertThat(bitmap.contains(10L)).isTrue(); + } + + @TestTemplate + public void testAddPositionsRequiringMultipleBitmaps() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + // construct positions that differ in their high 32-bit parts (i.e. keys) + long pos1 = ((long) 0 << 32) | 10L; // key = 0, low = 10 + long pos2 = ((long) 1 << 32) | 20L; // key = 1, low = 20 + long pos3 = ((long) 2 << 32) | 30L; // key = 2, low = 30 + long pos4 = ((long) 100 << 32) | 40L; // key = 100, low = 40 + + bitmap.set(pos1); + bitmap.set(pos2); + bitmap.set(pos3); + bitmap.set(pos4); + + assertThat(bitmap.contains(pos1)).isTrue(); + assertThat(bitmap.contains(pos2)).isTrue(); + assertThat(bitmap.contains(pos3)).isTrue(); + assertThat(bitmap.contains(pos4)).isTrue(); + assertThat(bitmap.cardinality()).isEqualTo(4); + assertThat(bitmap.serializedSizeInBytes()).isGreaterThan(4); + assertThat(bitmap.allocatedBitmapCount()).isEqualTo(101 /* max key + 1 */); + } + + @TestTemplate + public void testAddRange() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + long posStartInclusive = 10L; + long posEndExclusive = 20L; + bitmap.setRange(posStartInclusive, posEndExclusive); + + // assert that all positions in the range [10, 20) are added + for (long pos = posStartInclusive; pos < posEndExclusive; pos++) { + assertThat(bitmap.contains(pos)).isTrue(); + } + + // assert that positions outside the range are not present + assertThat(bitmap.contains(9L)).isFalse(); + assertThat(bitmap.contains(20L)).isFalse(); + + // assert that the cardinality is correct (10 positions in range [10, 20)) + assertThat(bitmap.cardinality()).isEqualTo(10); + } + + @TestTemplate + public void testAddRangeAcrossKeys() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + long posStartInclusive = ((long) 1 << 32) - 5L; + long posEndExclusive = ((long) 1 << 32) + 5L; + bitmap.setRange(posStartInclusive, posEndExclusive); + + // assert that all positions in the range are added + for (long pos = posStartInclusive; pos < posEndExclusive; pos++) { + assertThat(bitmap.contains(pos)).isTrue(); + } + + // assert that positions outside the range are not present + assertThat(bitmap.contains(0)).isFalse(); + assertThat(bitmap.contains(posEndExclusive)).isFalse(); + + // assert that the cardinality is correct + assertThat(bitmap.cardinality()).isEqualTo(10); + } + + @TestTemplate + public void testAddEmptyRange() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + bitmap.setRange(10, 10); + assertThat(bitmap.isEmpty()).isTrue(); + } + + @TestTemplate + public void testAddAll() { + RoaringPositionBitmap bitmap1 = new RoaringPositionBitmap(); + bitmap1.set(10L); + bitmap1.set(20L); + + RoaringPositionBitmap bitmap2 = new RoaringPositionBitmap(); + bitmap2.set(30L); + bitmap2.set(40L); + bitmap2.set((long) 2 << 32); + + bitmap1.setAll(bitmap2); + + assertThat(bitmap1.contains(10L)).isTrue(); + assertThat(bitmap1.contains(20L)).isTrue(); + assertThat(bitmap1.contains(30L)).isTrue(); + assertThat(bitmap1.contains(40L)).isTrue(); + assertThat(bitmap1.contains((long) 2 << 32)).isTrue(); + assertThat(bitmap1.cardinality()).isEqualTo(5); + + assertThat(bitmap2.contains(10L)).isFalse(); + assertThat(bitmap2.contains(20L)).isFalse(); + assertThat(bitmap2.cardinality()).isEqualTo(3); + } + + @TestTemplate + public void testAddAllWithEmptyBitmap() { + RoaringPositionBitmap bitmap1 = new RoaringPositionBitmap(); + bitmap1.set(10L); + bitmap1.set(20L); + + RoaringPositionBitmap emptyBitmap = new RoaringPositionBitmap(); + + bitmap1.setAll(emptyBitmap); + + assertThat(bitmap1.contains(10L)).isTrue(); + assertThat(bitmap1.contains(20L)).isTrue(); + assertThat(bitmap1.cardinality()).isEqualTo(2); + + assertThat(emptyBitmap.contains(10L)).isFalse(); + assertThat(emptyBitmap.contains(20L)).isFalse(); + assertThat(emptyBitmap.cardinality()).isEqualTo(0); + assertThat(emptyBitmap.isEmpty()).isTrue(); + } + + @TestTemplate + public void testAddAllWithOverlappingBitmap() { + RoaringPositionBitmap bitmap1 = new RoaringPositionBitmap(); + bitmap1.set(10L); + bitmap1.set(20L); + bitmap1.set(30L); + + RoaringPositionBitmap bitmap2 = new RoaringPositionBitmap(); + bitmap2.set(20L); + bitmap2.set(40L); + + bitmap1.setAll(bitmap2); + + assertThat(bitmap1.contains(10L)).isTrue(); + assertThat(bitmap1.contains(20L)).isTrue(); + assertThat(bitmap1.contains(30L)).isTrue(); + assertThat(bitmap1.contains(40L)).isTrue(); + assertThat(bitmap1.cardinality()).isEqualTo(4); + + assertThat(bitmap2.contains(10L)).isFalse(); + assertThat(bitmap2.contains(20L)).isTrue(); + assertThat(bitmap2.contains(30L)).isFalse(); + assertThat(bitmap2.contains(40L)).isTrue(); + assertThat(bitmap2.cardinality()).isEqualTo(2); + } + + @TestTemplate + public void testAddAllSparseBitmaps() { + RoaringPositionBitmap bitmap1 = new RoaringPositionBitmap(); + bitmap1.set((long) 0 << 32 | 100L); // key = 0, low = 100 + bitmap1.set((long) 1 << 32 | 200L); // key = 1, low = 200 + + RoaringPositionBitmap bitmap2 = new RoaringPositionBitmap(); + bitmap2.set((long) 2 << 32 | 300L); // key = 2, low = 300 + bitmap2.set((long) 3 << 32 | 400L); // key = 3, low = 400 + + bitmap1.setAll(bitmap2); + + assertThat(bitmap1.contains((long) 0 << 32 | 100L)).isTrue(); + assertThat(bitmap1.contains((long) 1 << 32 | 200L)).isTrue(); + assertThat(bitmap1.contains((long) 2 << 32 | 300L)).isTrue(); + assertThat(bitmap1.contains((long) 3 << 32 | 400L)).isTrue(); + assertThat(bitmap1.cardinality()).isEqualTo(4); + } + + @TestTemplate + public void testCardinality() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + assertThat(bitmap.cardinality()).isEqualTo(0); + + bitmap.set(10L); + bitmap.set(20L); + bitmap.set(30L); + + assertThat(bitmap.cardinality()).isEqualTo(3); + + bitmap.set(10L); // already exists + + assertThat(bitmap.cardinality()).isEqualTo(3); + } + + @TestTemplate + public void testCardinalitySparseBitmaps() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + bitmap.set((long) 0 << 32 | 100L); // key = 0, low = 100 + bitmap.set((long) 0 << 32 | 101L); // key = 0, low = 101 + bitmap.set((long) 0 << 32 | 105L); // key = 0, low = 101 + bitmap.set((long) 1 << 32 | 200L); // key = 1, low = 200 + bitmap.set((long) 100 << 32 | 300L); // key = 100, low = 300 + + assertThat(bitmap.cardinality()).isEqualTo(5); + } + + @TestTemplate + public void testSerializeDeserializeAllContainerBitmap() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + // bitmap 0, container 0 (array) + bitmap.set(position(0 /* bitmap */, 0 /* container */, 5L)); + bitmap.set(position(0 /* bitmap */, 0 /* container */, 7L)); + + // bitmap 0, container 1 (array that can be compressed) + bitmap.setRange( + position(0 /* bitmap */, 1 /* container */, 1L), + position(0 /* bitmap */, 1 /* container */, 1000L)); + + // bitmap 1, container 2 (bitset) + bitmap.setRange( + position(0 /* bitmap */, 2 /* container */, 1L), + position(0 /* bitmap */, 2 /* container */, CONTAINER_OFFSET - 1L)); + + // bitmap 1, container 0 (array) + bitmap.set(position(1 /* bitmap */, 0 /* container */, 10L)); + bitmap.set(position(1 /* bitmap */, 0 /* container */, 20L)); + + // bitmap 1, container 1 (array that can be compressed) + bitmap.setRange( + position(1 /* bitmap */, 1 /* container */, 10L), + position(1 /* bitmap */, 1 /* container */, 500L)); + + // bitmap 1, container 2 (bitset) + bitmap.setRange( + position(1 /* bitmap */, 2 /* container */, 1L), + position(1 /* bitmap */, 2 /* container */, CONTAINER_OFFSET - 1)); + + assertThat(bitmap.runLengthEncode()).as("Bitmap must be RLE encoded").isTrue(); + + RoaringPositionBitmap bitmapCopy = roundTripSerialize(bitmap); + + assertThat(bitmapCopy.cardinality()).isEqualTo(bitmap.cardinality()); + bitmapCopy.forEach(position -> assertThat(bitmap.contains(position)).isTrue()); + bitmap.forEach(position -> assertThat(bitmapCopy.contains(position)).isTrue()); + } + + @TestTemplate + public void testDeserializeSupportedRoaringExamples() throws IOException { + for (String file : SUPPORTED_OFFICIAL_EXAMPLE_FILES) { + RoaringPositionBitmap bitmap = readBitmap(file); + assertThat(bitmap).isNotNull(); + } + } + + @TestTemplate + public void testDeserializeUnsupportedRoaringExample() { + // this file contains a value that is larger than the max supported value in our impl + assertThatThrownBy(() -> readBitmap("64maphighvals.bin")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid unsigned key"); + } + + @TestTemplate + public void testUnsupportedPositions() { + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + + assertThatThrownBy(() -> bitmap.set(-1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bitmap supports positions that are >= 0 and <= %s", + RoaringPositionBitmap.MAX_POSITION); + + assertThatThrownBy(() -> bitmap.contains(-1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bitmap supports positions that are >= 0 and <= %s", + RoaringPositionBitmap.MAX_POSITION); + + assertThatThrownBy(() -> bitmap.set(RoaringPositionBitmap.MAX_POSITION + 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bitmap supports positions that are >= 0 and <= %s", + RoaringPositionBitmap.MAX_POSITION); + + assertThatThrownBy(() -> bitmap.contains(RoaringPositionBitmap.MAX_POSITION + 1L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bitmap supports positions that are >= 0 and <= %s", + RoaringPositionBitmap.MAX_POSITION); + } + + @TestTemplate + public void testInvalidSerializationByteOrder() { + assertThatThrownBy(() -> RoaringPositionBitmap.deserialize(ByteBuffer.allocate(4))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("serialization requires little-endian byte order"); + } + + @TestTemplate + public void testRandomSparseBitmap() { + Pair> bitmapAndPositions = + generateSparseBitmap( + 0L /* min position */, + (long) 5 << 32 /* max position must not need more than 5 bitmaps */, + 100_000 /* cardinality */); + RoaringPositionBitmap bitmap = bitmapAndPositions.first(); + Set positions = bitmapAndPositions.second(); + assertEqual(bitmap, positions); + assertRandomPositions(bitmap, positions); + } + + @TestTemplate + public void testRandomDenseBitmap() { + Pair> bitmapAndPositions = generateDenseBitmap(7); + RoaringPositionBitmap bitmap = bitmapAndPositions.first(); + Set positions = bitmapAndPositions.second(); + assertEqual(bitmap, positions); + assertRandomPositions(bitmap, positions); + } + + @TestTemplate + public void testRandomMixedBitmap() { + Pair> bitmapAndPositions = + generateSparseBitmap( + (long) 3 << 32 /* min position must need at least 3 bitmaps */, + (long) 5 << 32 /* max position must not need more than 5 bitmaps */, + 100_000 /* cardinality */); + RoaringPositionBitmap bitmap = bitmapAndPositions.first(); + Set positions = bitmapAndPositions.second(); + + Pair> pair1 = generateDenseBitmap(9); + bitmap.setAll(pair1.first()); + positions.addAll(pair1.second()); + + Pair> pair2 = + generateSparseBitmap( + 0 /* min position */, + (long) 3 << 32 /* max position must not need more than 3 bitmaps */, + 25_000 /* cardinality */); + bitmap.setAll(pair2.first()); + positions.addAll(pair2.second()); + + Pair> pair3 = generateDenseBitmap(3); + bitmap.setAll(pair3.first()); + positions.addAll(pair3.second()); + + Pair> pair4 = + generateSparseBitmap( + 0 /* min position */, + (long) 1 << 32 /* max position must not need more than 1 bitmap */, + 5_000 /* cardinality */); + bitmap.setAll(pair4.first()); + positions.addAll(pair4.second()); + + assertEqual(bitmap, positions); + assertRandomPositions(bitmap, positions); + } + + private Pair> generateSparseBitmap( + long minInclusive, long maxExclusive, int size) { + Random random = new Random(seed); + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + Set positions = Sets.newHashSet(); + + while (positions.size() < size) { + long position = nextLong(random, minInclusive, maxExclusive); + positions.add(position); + bitmap.set(position); + } + + return Pair.of(bitmap, positions); + } + + private Pair> generateDenseBitmap(int requiredBitmapCount) { + Random random = new Random(seed); + RoaringPositionBitmap bitmap = new RoaringPositionBitmap(); + Set positions = Sets.newHashSet(); + long currentPosition = 0; + + while (bitmap.allocatedBitmapCount() <= requiredBitmapCount) { + long maxRunPosition = currentPosition + nextLong(random, 1000, 2 * CONTAINER_SIZE); + for (long position = currentPosition; position <= maxRunPosition; position++) { + bitmap.set(position); + positions.add(position); + } + long shift = nextLong(random, (long) (0.1 * BITMAP_SIZE), (long) (0.25 * BITMAP_SIZE)); + currentPosition = maxRunPosition + shift; + } + + return Pair.of(bitmap, positions); + } + + private void assertRandomPositions(RoaringPositionBitmap bitmap, Set positions) { + Random random = new Random(validationSeed); + for (int ordinal = 0; ordinal < VALIDATION_LOOKUP_COUNT; ordinal++) { + long position = nextLong(random, 0, RoaringPositionBitmap.MAX_POSITION); + assertThat(bitmap.contains(position)).isEqualTo(positions.contains(position)); + } + } + + private static long nextLong(Random random, long minInclusive, long maxExclusive) { + return minInclusive + (long) (random.nextDouble() * (maxExclusive - minInclusive)); + } + + private static long position(int bitmapIndex, int containerIndex, long value) { + return bitmapIndex * BITMAP_OFFSET + containerIndex * CONTAINER_OFFSET + value; + } + + private static RoaringPositionBitmap roundTripSerialize(RoaringPositionBitmap bitmap) { + ByteBuffer buffer = ByteBuffer.allocate((int) bitmap.serializedSizeInBytes()); + buffer.order(ByteOrder.LITTLE_ENDIAN); + bitmap.serialize(buffer); + buffer.flip(); + return RoaringPositionBitmap.deserialize(buffer); + } + + private static RoaringPositionBitmap readBitmap(String resourceName) throws IOException { + byte[] bytes = readTestResource(resourceName); + ByteBuffer buffer = ByteBuffer.wrap(bytes); + buffer.order(ByteOrder.LITTLE_ENDIAN); + return RoaringPositionBitmap.deserialize(buffer); + } + + private static byte[] readTestResource(String resourceName) throws IOException { + URL resource = Resources.getResource(TestRoaringPositionBitmap.class, resourceName); + return Resources.toByteArray(resource); + } + + private static void assertEqual(RoaringPositionBitmap bitmap, Set positions) { + assertEqualContent(bitmap, positions); + + RoaringPositionBitmap bitmapCopy1 = roundTripSerialize(bitmap); + assertEqualContent(bitmapCopy1, positions); + + bitmap.runLengthEncode(); + RoaringPositionBitmap bitmapCopy2 = roundTripSerialize(bitmap); + assertEqualContent(bitmapCopy2, positions); + } + + private static void assertEqualContent(RoaringPositionBitmap bitmap, Set positions) { + assertThat(bitmap.cardinality()).isEqualTo(positions.size()); + positions.forEach(position -> assertThat(bitmap.contains(position)).isTrue()); + bitmap.forEach(position -> assertThat(positions.contains(position)).isTrue()); + } +} diff --git a/core/src/test/resources/org/apache/iceberg/deletes/64map32bitvals.bin b/core/src/test/resources/org/apache/iceberg/deletes/64map32bitvals.bin new file mode 100644 index 0000000000000000000000000000000000000000..475b894417e44cff61d8810057fc1530cef05718 GIT binary patch literal 48 ocmZQ%KmaQP1_nkjmy9 literal 0 HcmV?d00001 diff --git a/core/src/test/resources/org/apache/iceberg/deletes/64maphighvals.bin b/core/src/test/resources/org/apache/iceberg/deletes/64maphighvals.bin new file mode 100644 index 0000000000000000000000000000000000000000..d4312b8d22713991026a36d5d1293cf1960d89ed GIT binary patch literal 1086 zcmd;PfPnY=_rj5t0RsagP#7Y>#UKD@!S*SERgUMnOxf@r{zi~~v PF5QrBO1Grj(uH&%!J7vn literal 0 HcmV?d00001 From 602c2b2dbecb81d7d84940f988579add7ffd1030 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 29 Oct 2024 17:54:24 +0100 Subject: [PATCH 067/313] Flink 1.20: Update Flink to use planned Avro reads (#11386) --- .../iceberg/flink/data/FlinkAvroReader.java | 12 ++ .../flink/data/FlinkPlannedAvroReader.java | 192 ++++++++++++++++++ .../iceberg/flink/data/FlinkValueReaders.java | 32 +++ .../source/RowDataFileScanTaskReader.java | 4 +- .../flink/data/TestFlinkAvroReaderWriter.java | 2 +- .../iceberg/flink/data/TestRowProjection.java | 2 +- 6 files changed, 240 insertions(+), 4 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java index 86404959735a..baae91dd1882 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java @@ -37,16 +37,28 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +/** + * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. + */ +@Deprecated public class FlinkAvroReader implements DatumReader, SupportsRowPosition { private final Schema readSchema; private final ValueReader reader; private Schema fileSchema = null; + /** + * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. + */ + @Deprecated public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { this(expectedSchema, readSchema, ImmutableMap.of()); } + /** + * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. + */ + @Deprecated @SuppressWarnings("unchecked") public FlinkAvroReader( org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java new file mode 100644 index 000000000000..b7a81752d4a0 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.avro.AvroWithPartnerVisitor; +import org.apache.iceberg.avro.SupportsRowPosition; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; + +public class FlinkPlannedAvroReader implements DatumReader, SupportsRowPosition { + + private final Types.StructType expectedType; + private final Map idToConstant; + private ValueReader reader; + + public static FlinkPlannedAvroReader create(org.apache.iceberg.Schema schema) { + return create(schema, ImmutableMap.of()); + } + + public static FlinkPlannedAvroReader create( + org.apache.iceberg.Schema schema, Map constants) { + return new FlinkPlannedAvroReader(schema, constants); + } + + private FlinkPlannedAvroReader( + org.apache.iceberg.Schema expectedSchema, Map constants) { + this.expectedType = expectedSchema.asStruct(); + this.idToConstant = constants; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema fileSchema) { + this.reader = + (ValueReader) + AvroWithPartnerVisitor.visit( + expectedType, + fileSchema, + new ReadBuilder(idToConstant), + AvroWithPartnerVisitor.FieldIDAccessors.get()); + } + + @Override + public RowData read(RowData reuse, Decoder decoder) throws IOException { + return reader.read(decoder, reuse); + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + private static class ReadBuilder extends AvroWithPartnerVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public ValueReader record(Type partner, Schema record, List> fieldReaders) { + if (partner == null) { + return ValueReaders.skipStruct(fieldReaders); + } + + Types.StructType expected = partner.asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldReaders, idToConstant); + + // TODO: should this pass expected so that struct.get can reuse containers? + return FlinkValueReaders.struct(readPlan, expected.fields().size()); + } + + @Override + public ValueReader union(Type partner, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader array(Type partner, Schema array, ValueReader elementReader) { + return FlinkValueReaders.array(elementReader); + } + + @Override + public ValueReader arrayMap( + Type partner, Schema map, ValueReader keyReader, ValueReader valueReader) { + return FlinkValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader map(Type partner, Schema map, ValueReader valueReader) { + return FlinkValueReaders.map(FlinkValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Type partner, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + // Flink uses the same representation + return ValueReaders.ints(); + + case "time-micros": + return FlinkValueReaders.timeMicros(); + + case "timestamp-millis": + return FlinkValueReaders.timestampMills(); + + case "timestamp-micros": + return FlinkValueReaders.timestampMicros(); + + case "decimal": + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return FlinkValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + decimal.getPrecision(), + decimal.getScale()); + + case "uuid": + return FlinkValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType.getName()); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + if (partner != null && partner.typeId() == Type.TypeID.LONG) { + return ValueReaders.intsAsLongs(); + } + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + if (partner != null && partner.typeId() == Type.TypeID.DOUBLE) { + return ValueReaders.floatsAsDoubles(); + } + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return FlinkValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive.getFixedSize()); + case BYTES: + return ValueReaders.bytes(); + case ENUM: + return FlinkValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java index 32f6c3a2ccfd..0c6ff2411160 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; public class FlinkValueReaders { @@ -86,6 +87,10 @@ static ValueReader map(ValueReader keyReader, ValueReader valueRe return new MapReader(keyReader, valueReader); } + static ValueReader struct(List>> readPlan, int numFields) { + return new PlannedStructReader(readPlan, numFields); + } + static ValueReader struct( List> readers, Types.StructType struct, Map idToConstant) { return new StructReader(readers, struct, idToConstant); @@ -282,6 +287,33 @@ public MapData read(Decoder decoder, Object reuse) throws IOException { } } + private static class PlannedStructReader extends ValueReaders.PlannedStructReader { + private final int numFields; + + private PlannedStructReader(List>> readPlan, int numFields) { + super(readPlan); + this.numFields = numFields; + } + + @Override + protected RowData reuseOrCreate(Object reuse) { + if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) { + return (RowData) reuse; + } + return new GenericRowData(numFields); + } + + @Override + protected Object get(RowData struct, int pos) { + return null; + } + + @Override + protected void set(RowData struct, int pos, Object value) { + ((GenericRowData) struct).setField(pos, value); + } + } + private static class StructReader extends ValueReaders.StructReader { private final int numFields; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index 88364f4e87b1..9c75a5e0f0fc 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -35,9 +35,9 @@ import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkSourceFilter; import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.FlinkAvroReader; import org.apache.iceberg.flink.data.FlinkOrcReader; import org.apache.iceberg.flink.data.FlinkParquetReaders; +import org.apache.iceberg.flink.data.FlinkPlannedAvroReader; import org.apache.iceberg.flink.data.RowDataProjection; import org.apache.iceberg.flink.data.RowDataUtil; import org.apache.iceberg.io.CloseableIterable; @@ -154,7 +154,7 @@ private CloseableIterable newAvroIterable( .reuseContainers() .project(schema) .split(task.start(), task.length()) - .createReaderFunc(readSchema -> new FlinkAvroReader(schema, readSchema, idToConstant)); + .createReaderFunc(readSchema -> FlinkPlannedAvroReader.create(schema, idToConstant)); if (nameMapping != null) { builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java index a1039d27d888..2b9e8694b6d6 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java @@ -91,7 +91,7 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n try (CloseableIterable reader = Avro.read(Files.localInput(recordsFile)) .project(schema) - .createReaderFunc(FlinkAvroReader::new) + .createResolvingReader(FlinkPlannedAvroReader::create) .build()) { Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index 7dd4e8759c0e..3b6cf0c58f8f 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -64,7 +64,7 @@ private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, Iterable records = Avro.read(Files.localInput(file)) .project(readSchema) - .createReaderFunc(FlinkAvroReader::new) + .createResolvingReader(FlinkPlannedAvroReader::create) .build(); return Iterables.getOnlyElement(records); From 740d4e7b1ced2c1b4549edf9f6189003f0e06c2c Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Tue, 29 Oct 2024 23:49:10 +0530 Subject: [PATCH 068/313] open-api: Fix `testFixtures` dependencies (#11422) --- build.gradle | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index abab68ca4b8b..fe8392279e62 100644 --- a/build.gradle +++ b/build.gradle @@ -963,10 +963,28 @@ project(':iceberg-open-api') { testFixturesImplementation project(':iceberg-api') testFixturesImplementation project(':iceberg-core') testFixturesImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') - testFixturesImplementation project(':iceberg-core').sourceSets.test.runtimeClasspath testFixturesImplementation project(':iceberg-aws') testFixturesImplementation project(':iceberg-gcp') testFixturesImplementation project(':iceberg-azure') + testFixturesImplementation(libs.hadoop3.common) { + exclude group: 'log4j' + exclude group: 'org.slf4j' + exclude group: 'ch.qos.reload4j' + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'com.fasterxml.woodstox' + exclude group: 'com.google.guava' + exclude group: 'com.google.protobuf' + exclude group: 'org.apache.curator' + exclude group: 'org.apache.zookeeper' + exclude group: 'org.apache.kerby' + exclude group: 'org.apache.hadoop', module: 'hadoop-auth' + exclude group: 'org.apache.commons', module: 'commons-configuration2' + exclude group: 'org.apache.hadoop.thirdparty', module: 'hadoop-shaded-protobuf_3_7' + exclude group: 'org.codehaus.woodstox' + exclude group: 'org.eclipse.jetty' + } + testFixturesImplementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + testFixturesImplementation libs.junit.jupiter testFixturesImplementation libs.jetty.servlet testFixturesImplementation libs.jetty.server From 469c5560b26f8c0eb18bd04be8f4d8fb1fd87c20 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Tue, 29 Oct 2024 13:03:38 -0700 Subject: [PATCH 069/313] Core: use ManifestFiles.open when possible (#11414) --- .../main/java/org/apache/iceberg/BaseFilesTable.java | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java index b71744f739c7..4dff19b87990 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java @@ -181,15 +181,7 @@ public long estimatedRowsCount() { } private CloseableIterable> files(Schema fileProjection) { - switch (manifest.content()) { - case DATA: - return ManifestFiles.read(manifest, io, specsById).project(fileProjection); - case DELETES: - return ManifestFiles.readDeleteManifest(manifest, io, specsById).project(fileProjection); - default: - throw new IllegalArgumentException( - "Unsupported manifest content type:" + manifest.content()); - } + return ManifestFiles.open(manifest, io, specsById).project(fileProjection); } /** From 5359bea71bceaad35897ac82573d73e3c7e47f71 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 30 Oct 2024 07:09:42 +0100 Subject: [PATCH 070/313] GCS: Refresh vended credentials (#11282) --- build.gradle | 3 + .../org/apache/iceberg/gcp/GCPProperties.java | 20 ++ .../org/apache/iceberg/gcp/gcs/GCSFileIO.java | 12 +- .../gcs/OAuth2RefreshCredentialsHandler.java | 99 +++++++ .../apache/iceberg/gcp/GCPPropertiesTest.java | 30 ++ .../apache/iceberg/gcp/gcs/GCSFileIOTest.java | 47 ++++ .../OAuth2RefreshCredentialsHandlerTest.java | 264 ++++++++++++++++++ 7 files changed, 474 insertions(+), 1 deletion(-) create mode 100644 gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java create mode 100644 gcp/src/test/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandlerTest.java diff --git a/build.gradle b/build.gradle index fe8392279e62..1a05f83f7d9b 100644 --- a/build.gradle +++ b/build.gradle @@ -644,6 +644,7 @@ project(':iceberg-gcp') { testImplementation "com.google.cloud:google-cloud-nio" testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation(libs.hadoop2.common) { exclude group: 'org.apache.avro', module: 'avro' @@ -652,6 +653,8 @@ project(':iceberg-gcp') { exclude group: 'com.google.code.gson', module: 'gson' } testImplementation libs.esotericsoftware.kryo + testImplementation libs.mockserver.netty + testImplementation libs.mockserver.client.java } } diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java index 4f60e2f91f91..c03906ae5d1e 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -43,6 +43,12 @@ public class GCPProperties implements Serializable { public static final String GCS_OAUTH2_TOKEN_EXPIRES_AT = "gcs.oauth2.token-expires-at"; // Boolean to explicitly configure "no authentication" for testing purposes using a GCS emulator public static final String GCS_NO_AUTH = "gcs.no-auth"; + public static final String GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT = + "gcs.oauth2.refresh-credentials-endpoint"; + + /** Controls whether vended credentials should be refreshed or not. Defaults to true. */ + public static final String GCS_OAUTH2_REFRESH_CREDENTIALS_ENABLED = + "gcs.oauth2.refresh-credentials-enabled"; /** Configure the batch size used when deleting multiple files from a given GCS bucket */ public static final String GCS_DELETE_BATCH_SIZE = "gcs.delete.batch-size"; @@ -67,6 +73,8 @@ public class GCPProperties implements Serializable { private boolean gcsNoAuth; private String gcsOAuth2Token; private Date gcsOAuth2TokenExpiresAt; + private String gcsOauth2RefreshCredentialsEndpoint; + private boolean gcsOauth2RefreshCredentialsEnabled; private int gcsDeleteBatchSize = GCS_DELETE_BATCH_SIZE_DEFAULT; @@ -95,6 +103,10 @@ public GCPProperties(Map properties) { gcsOAuth2TokenExpiresAt = new Date(Long.parseLong(properties.get(GCS_OAUTH2_TOKEN_EXPIRES_AT))); } + + gcsOauth2RefreshCredentialsEndpoint = properties.get(GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT); + gcsOauth2RefreshCredentialsEnabled = + PropertyUtil.propertyAsBoolean(properties, GCS_OAUTH2_REFRESH_CREDENTIALS_ENABLED, true); gcsNoAuth = Boolean.parseBoolean(properties.getOrDefault(GCS_NO_AUTH, "false")); Preconditions.checkState( !(gcsOAuth2Token != null && gcsNoAuth), @@ -154,4 +166,12 @@ public Optional oauth2TokenExpiresAt() { public int deleteBatchSize() { return gcsDeleteBatchSize; } + + public Optional oauth2RefreshCredentialsEndpoint() { + return Optional.ofNullable(gcsOauth2RefreshCredentialsEndpoint); + } + + public boolean oauth2RefreshCredentialsEnabled() { + return gcsOauth2RefreshCredentialsEnabled; + } } diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java index 2201c876bd38..5737606aef5e 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java @@ -20,6 +20,7 @@ import com.google.auth.oauth2.AccessToken; import com.google.auth.oauth2.OAuth2Credentials; +import com.google.auth.oauth2.OAuth2CredentialsWithRefresh; import com.google.cloud.NoCredentials; import com.google.cloud.storage.Blob; import com.google.cloud.storage.BlobId; @@ -156,7 +157,16 @@ public void initialize(Map props) { // Explicitly configure an OAuth token. AccessToken accessToken = new AccessToken(token, gcpProperties.oauth2TokenExpiresAt().orElse(null)); - builder.setCredentials(OAuth2Credentials.create(accessToken)); + if (gcpProperties.oauth2RefreshCredentialsEnabled() + && gcpProperties.oauth2RefreshCredentialsEndpoint().isPresent()) { + builder.setCredentials( + OAuth2CredentialsWithRefresh.newBuilder() + .setAccessToken(accessToken) + .setRefreshHandler(OAuth2RefreshCredentialsHandler.create(properties)) + .build()); + } else { + builder.setCredentials(OAuth2Credentials.create(accessToken)); + } }); return builder.build().getService(); diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java new file mode 100644 index 000000000000..611e7baaec6e --- /dev/null +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java @@ -0,0 +1,99 @@ +/* + * 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.gcp.gcs; + +import com.google.auth.oauth2.AccessToken; +import com.google.auth.oauth2.OAuth2CredentialsWithRefresh; +import java.io.IOException; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.gcp.GCPProperties; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.ErrorHandlers; +import org.apache.iceberg.rest.HTTPClient; +import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.auth.OAuth2Properties; +import org.apache.iceberg.rest.auth.OAuth2Util; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.responses.LoadCredentialsResponse; + +public class OAuth2RefreshCredentialsHandler + implements OAuth2CredentialsWithRefresh.OAuth2RefreshHandler { + private final Map properties; + + private OAuth2RefreshCredentialsHandler(Map properties) { + Preconditions.checkArgument( + null != properties.get(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT), + "Invalid credentials endpoint: null"); + this.properties = properties; + } + + @Override + public AccessToken refreshAccessToken() { + LoadCredentialsResponse response; + try (RESTClient client = httpClient()) { + response = + client.get( + properties.get(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT), + null, + LoadCredentialsResponse.class, + OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)), + ErrorHandlers.defaultErrorHandler()); + } catch (IOException e) { + throw new RuntimeException(e); + } + + List gcsCredentials = + response.credentials().stream() + .filter(c -> c.prefix().startsWith("gs")) + .collect(Collectors.toList()); + + Preconditions.checkState(!gcsCredentials.isEmpty(), "Invalid GCS Credentials: empty"); + Preconditions.checkState( + gcsCredentials.size() == 1, + "Invalid GCS Credentials: only one GCS credential should exist"); + + Credential gcsCredential = gcsCredentials.get(0); + checkCredential(gcsCredential, GCPProperties.GCS_OAUTH2_TOKEN); + checkCredential(gcsCredential, GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT); + String token = gcsCredential.config().get(GCPProperties.GCS_OAUTH2_TOKEN); + String expiresAt = gcsCredential.config().get(GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT); + + return new AccessToken(token, new Date(Long.parseLong(expiresAt))); + } + + private void checkCredential(Credential gcsCredential, String gcsOauth2Token) { + Preconditions.checkState( + gcsCredential.config().containsKey(gcsOauth2Token), + "Invalid GCS Credentials: %s not set", + gcsOauth2Token); + } + + public static OAuth2RefreshCredentialsHandler create(Map properties) { + return new OAuth2RefreshCredentialsHandler(properties); + } + + private RESTClient httpClient() { + return HTTPClient.builder(properties) + .uri(properties.get(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT)) + .build(); + } +} diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java b/gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java index c71b55828758..61bd069f0c27 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java @@ -19,6 +19,8 @@ package org.apache.iceberg.gcp; import static org.apache.iceberg.gcp.GCPProperties.GCS_NO_AUTH; +import static org.apache.iceberg.gcp.GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENABLED; +import static org.apache.iceberg.gcp.GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT; import static org.apache.iceberg.gcp.GCPProperties.GCS_OAUTH2_TOKEN; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalStateException; @@ -47,4 +49,32 @@ public void testOAuthWithNoAuth() { assertThat(gcpProperties.noAuth()).isTrue(); assertThat(gcpProperties.oauth2Token()).isNotPresent(); } + + @Test + public void refreshCredentialsEndpointSet() { + GCPProperties gcpProperties = + new GCPProperties( + ImmutableMap.of(GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, "/v1/credentials")); + assertThat(gcpProperties.oauth2RefreshCredentialsEnabled()).isTrue(); + assertThat(gcpProperties.oauth2RefreshCredentialsEndpoint()) + .isPresent() + .get() + .isEqualTo("/v1/credentials"); + } + + @Test + public void refreshCredentialsEndpointSetButRefreshDisabled() { + GCPProperties gcpProperties = + new GCPProperties( + ImmutableMap.of( + GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, + "/v1/credentials", + GCS_OAUTH2_REFRESH_CREDENTIALS_ENABLED, + "false")); + assertThat(gcpProperties.oauth2RefreshCredentialsEnabled()).isFalse(); + assertThat(gcpProperties.oauth2RefreshCredentialsEndpoint()) + .isPresent() + .get() + .isEqualTo("/v1/credentials"); + } } diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java index fbc3fe7114bb..6302f664b70a 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java @@ -19,11 +19,17 @@ package org.apache.iceberg.gcp.gcs; import static java.lang.String.format; +import static org.apache.iceberg.gcp.GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENABLED; +import static org.apache.iceberg.gcp.GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT; +import static org.apache.iceberg.gcp.GCPProperties.GCS_OAUTH2_TOKEN; +import static org.apache.iceberg.gcp.GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; +import com.google.auth.oauth2.OAuth2Credentials; +import com.google.auth.oauth2.OAuth2CredentialsWithRefresh; import com.google.cloud.storage.BlobId; import com.google.cloud.storage.BlobInfo; import com.google.cloud.storage.Storage; @@ -32,6 +38,8 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.List; import java.util.Random; import java.util.stream.StreamSupport; @@ -223,4 +231,43 @@ public void testResolvingFileIOLoad() { .invoke("gs://foo/bar"); assertThat(result).isInstanceOf(GCSFileIO.class); } + + @Test + public void refreshCredentialsEndpointSet() { + Storage client; + try (GCSFileIO fileIO = new GCSFileIO()) { + fileIO.initialize( + ImmutableMap.of( + GCS_OAUTH2_TOKEN, + "gcsToken", + GCS_OAUTH2_TOKEN_EXPIRES_AT, + Long.toString(Instant.now().plus(5, ChronoUnit.MINUTES).toEpochMilli()), + GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, + "/v1/credentials")); + client = fileIO.client(); + } + + assertThat(client.getOptions().getCredentials()) + .isInstanceOf(OAuth2CredentialsWithRefresh.class); + } + + @Test + public void refreshCredentialsEndpointSetButRefreshDisabled() { + Storage client; + try (GCSFileIO fileIO = new GCSFileIO()) { + fileIO.initialize( + ImmutableMap.of( + GCS_OAUTH2_TOKEN, + "gcsTokenWithoutRefresh", + GCS_OAUTH2_TOKEN_EXPIRES_AT, + Long.toString(Instant.now().plus(5, ChronoUnit.MINUTES).toEpochMilli()), + GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, + "/v1/credentials", + GCS_OAUTH2_REFRESH_CREDENTIALS_ENABLED, + "false")); + client = fileIO.client(); + } + + assertThat(client.getOptions().getCredentials()).isInstanceOf(OAuth2Credentials.class); + } } diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandlerTest.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandlerTest.java new file mode 100644 index 000000000000..c538745f2767 --- /dev/null +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandlerTest.java @@ -0,0 +1,264 @@ +/* + * 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.gcp.gcs; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockserver.integration.ClientAndServer.startClientAndServer; + +import com.google.auth.oauth2.AccessToken; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import org.apache.iceberg.exceptions.BadRequestException; +import org.apache.iceberg.exceptions.RESTException; +import org.apache.iceberg.gcp.GCPProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.HttpMethod; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.ImmutableCredential; +import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponseParser; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockserver.integration.ClientAndServer; +import org.mockserver.model.HttpRequest; +import org.mockserver.model.HttpResponse; +import org.mockserver.verify.VerificationTimes; + +public class OAuth2RefreshCredentialsHandlerTest { + private static final int PORT = 3333; + private static final String URI = String.format("http://127.0.0.1:%d/v1/credentials", PORT); + private static ClientAndServer mockServer; + + @BeforeAll + public static void beforeAll() { + mockServer = startClientAndServer(PORT); + } + + @AfterAll + public static void stopServer() { + mockServer.stop(); + } + + @BeforeEach + public void before() { + mockServer.reset(); + } + + @Test + public void invalidOrMissingUri() { + assertThatThrownBy(() -> OAuth2RefreshCredentialsHandler.create(ImmutableMap.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid credentials endpoint: null"); + + assertThatThrownBy( + () -> + OAuth2RefreshCredentialsHandler.create( + ImmutableMap.of( + GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, "invalid uri")) + .refreshAccessToken()) + .isInstanceOf(RESTException.class) + .hasMessageStartingWith("Failed to create request URI from base invalid uri"); + } + + @Test + public void badRequest() { + HttpRequest mockRequest = + HttpRequest.request("/v1/credentials").withMethod(HttpMethod.GET.name()); + + HttpResponse mockResponse = HttpResponse.response().withStatusCode(400); + mockServer.when(mockRequest).respond(mockResponse); + + OAuth2RefreshCredentialsHandler handler = + OAuth2RefreshCredentialsHandler.create( + ImmutableMap.of(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, URI)); + + assertThatThrownBy(handler::refreshAccessToken) + .isInstanceOf(BadRequestException.class) + .hasMessageStartingWith("Malformed request"); + } + + @Test + public void noGcsCredentialInResponse() { + HttpRequest mockRequest = + HttpRequest.request("/v1/credentials").withMethod(HttpMethod.GET.name()); + + HttpResponse mockResponse = + HttpResponse.response( + LoadCredentialsResponseParser.toJson( + ImmutableLoadCredentialsResponse.builder().build())) + .withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + OAuth2RefreshCredentialsHandler handler = + OAuth2RefreshCredentialsHandler.create( + ImmutableMap.of(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, URI)); + + assertThatThrownBy(handler::refreshAccessToken) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid GCS Credentials: empty"); + } + + @Test + public void noGcsToken() { + HttpRequest mockRequest = + HttpRequest.request("/v1/credentials").withMethod(HttpMethod.GET.name()); + + Credential credential = + ImmutableCredential.builder() + .prefix("gs") + .config(ImmutableMap.of(GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT, "1000")) + .build(); + HttpResponse mockResponse = + HttpResponse.response( + LoadCredentialsResponseParser.toJson( + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build())) + .withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + OAuth2RefreshCredentialsHandler handler = + OAuth2RefreshCredentialsHandler.create( + ImmutableMap.of(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, URI)); + + assertThatThrownBy(handler::refreshAccessToken) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid GCS Credentials: gcs.oauth2.token not set"); + } + + @Test + public void tokenWithoutExpiration() { + HttpRequest mockRequest = + HttpRequest.request("/v1/credentials").withMethod(HttpMethod.GET.name()); + + Credential credential = + ImmutableCredential.builder() + .prefix("gs") + .config(ImmutableMap.of(GCPProperties.GCS_OAUTH2_TOKEN, "gcsToken")) + .build(); + HttpResponse mockResponse = + HttpResponse.response( + LoadCredentialsResponseParser.toJson( + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build())) + .withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + OAuth2RefreshCredentialsHandler handler = + OAuth2RefreshCredentialsHandler.create( + ImmutableMap.of(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, URI)); + + assertThatThrownBy(handler::refreshAccessToken) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid GCS Credentials: gcs.oauth2.token-expires-at not set"); + } + + @Test + public void tokenWithExpiration() { + HttpRequest mockRequest = + HttpRequest.request("/v1/credentials").withMethod(HttpMethod.GET.name()); + + Credential credential = + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + GCPProperties.GCS_OAUTH2_TOKEN, + "gcsToken", + GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT, + Long.toString(Instant.now().plus(5, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + HttpResponse mockResponse = + HttpResponse.response( + LoadCredentialsResponseParser.toJson( + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build())) + .withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + OAuth2RefreshCredentialsHandler handler = + OAuth2RefreshCredentialsHandler.create( + ImmutableMap.of(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, URI)); + + AccessToken accessToken = handler.refreshAccessToken(); + assertThat(accessToken.getTokenValue()) + .isEqualTo(credential.config().get(GCPProperties.GCS_OAUTH2_TOKEN)); + assertThat(accessToken.getExpirationTime().toInstant().toEpochMilli()) + .isEqualTo( + Long.parseLong(credential.config().get(GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT))); + + // refresh always fetches a new token + AccessToken refreshedToken = handler.refreshAccessToken(); + assertThat(refreshedToken).isNotSameAs(accessToken); + + mockServer.verify(mockRequest, VerificationTimes.exactly(2)); + } + + @Test + public void multipleGcsCredentials() { + HttpRequest mockRequest = + HttpRequest.request("/v1/credentials").withMethod(HttpMethod.GET.name()); + + Credential credentialOne = + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + GCPProperties.GCS_OAUTH2_TOKEN, + "gcsToken1", + GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT, + Long.toString(Instant.now().plus(1, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + Credential credentialTwo = + ImmutableCredential.builder() + .prefix("gs://my-custom-prefix/xyz/long-prefix") + .config( + ImmutableMap.of( + GCPProperties.GCS_OAUTH2_TOKEN, + "gcsToken2", + GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT, + Long.toString(Instant.now().plus(2, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + Credential credentialThree = + ImmutableCredential.builder() + .prefix("gs://my-custom-prefix/xyz") + .config( + ImmutableMap.of( + GCPProperties.GCS_OAUTH2_TOKEN, + "gcsToken3", + GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT, + Long.toString(Instant.now().plus(3, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + HttpResponse mockResponse = + HttpResponse.response( + LoadCredentialsResponseParser.toJson( + ImmutableLoadCredentialsResponse.builder() + .addCredentials(credentialOne, credentialTwo, credentialThree) + .build())) + .withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + OAuth2RefreshCredentialsHandler handler = + OAuth2RefreshCredentialsHandler.create( + ImmutableMap.of(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT, URI)); + + assertThatThrownBy(handler::refreshAccessToken) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid GCS Credentials: only one GCS credential should exist"); + } +} From bedc71167b8c7d6a9f167ee373794f02fb968cd7 Mon Sep 17 00:00:00 2001 From: Alex Merced Date: Wed, 30 Oct 2024 02:26:10 -0400 Subject: [PATCH 071/313] Docs: Add 21 blogs / fix one broken link (#11424) --- site/docs/blogs.md | 133 ++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 132 insertions(+), 1 deletion(-) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 819d4f8b9e92..0d29ebcec56c 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -22,6 +22,83 @@ title: "Blogs" Here is a list of company blogs that talk about Iceberg. The blogs are ordered from most recent to oldest. + +### [Hands-on with Apache Iceberg Tables using PyIceberg using Nessie and Minio](https://www.dremio.com/blog/intro-to-pyiceberg/) +**Date**: October 22nd, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [A Brief Guide to the Governance of Apache Iceberg Tables](https://medium.com/data-engineering-with-dremio/a-brief-guide-to-the-governance-of-apache-iceberg-tables-7c0a50316e22) +**Date**: October 8th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Ultimate Directory of Apache Iceberg Resources](https://medium.com/data-engineering-with-dremio/ultimate-directory-of-apache-iceberg-resources-e3e02efac62e) +**Date**: October 7th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [A Guide to Change Data Capture (CDC) with Apache Iceberg](https://www.dremio.com/blog/cdc-with-apache-iceberg/) +**Date**: October 3rd, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Using Nessie’s REST Catalog Support for Working with Apache Iceberg Tables](https://www.dremio.com/blog/use-nessie-with-iceberg-rest-catalog/) +**Date**: October 3rd, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Hands-on with Apache Iceberg on Your Laptop: Deep Dive with Apache Spark, Nessie, Minio, Dremio, Polars and Seaborn](https://medium.com/data-engineering-with-dremio/hands-on-with-apache-iceberg-on-your-laptop-deep-dive-with-apache-spark-nessie-minio-dremio-c5d689b01730) +**Date**: September 20th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Leveraging Apache Iceberg Metadata Tables in Dremio for Effective Data Lakehouse Auditing](https://www.dremio.com/blog/apache-iceberg-metadata-tables/) +**Date**: September 16th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Why Thinking about Apache Iceberg Catalogs Like Nessie and Apache Polaris (incubating) Matters](https://www.dremio.com/blog/why-thinking-about-apache-iceberg-catalogs-like-nessie-and-apache-polaris-incubating-matters/) +**Date**: September 5th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [8 Tools For Ingesting Data Into Apache Iceberg](https://www.dremio.com/blog/8-tools-for-ingesting-data-into-apache-iceberg/) +**Date**: August 20th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Evolving the Data Lake: From CSV/JSON to Parquet to Apache Iceberg](https://www.dremio.com/blog/evolving-the-data-lake-from-csv-json-to-parquet-to-apache-iceberg/) +**Date**: August 19th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Guide to Maintaining an Apache Iceberg Lakehouse](https://www.dremio.com/blog/guide-to-maintaining-an-apache-iceberg-lakehouse/) +**Date**: August 12th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Migration Guide for Apache Iceberg Lakehouses](https://www.dremio.com/blog/migration-guide-for-apache-iceberg-lakehouses/) +**Date**: August 8th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Getting Hands-on with Polaris OSS, Apache Iceberg and Apache Spark](https://www.dremio.com/blog/getting-hands-on-with-polaris-oss-apache-iceberg-and-apache-spark/) +**Date**: August 1st, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) ### [Sending Data to Apache Iceberg from Apache Kafka with Apache Flink](https://www.decodable.co/blog/kafka-to-iceberg-with-flink) @@ -29,6 +106,12 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Robin Moffatt](https://www.linkedin.com/in/robinmoffatt) + +### [What is a Data Lakehouse and a Table Format?](https://www.dremio.com/blog/apache-iceberg-crash-course-what-is-a-data-lakehouse-and-a-table-format/) +**Date**: July 11th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [How to get data from Apache Kafka to Apache Iceberg on S3 with Decodable](https://www.decodable.co/blog/kafka-to-iceberg-with-decodable) **Date**: June 18th, 2024, **Company**: Decodable @@ -36,7 +119,55 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Robin Moffatt](https://www.linkedin.com/in/robinmoffatt) -### [End-to-End Basic Data Engineering Tutorial (Apache Spark, Apache Iceberg, Dremio, Apache Superset, Nessie)](https://medium.com/data-engineering-with-dremio/end-to-end-basic-data-engineering-tutorial-apache-spark-apache-iceberg-dremio-apache-superset-a896ecab46f6) +### [The Nessie Ecosystem and the Reach of Git for Data for Apache Iceberg](https://www.dremio.com/blog/the-nessie-ecosystem-and-the-reach-of-git-for-data-for-apache-iceberg/) +**Date**: May 28th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [The Evolution of Apache Iceberg Catalogs](https://www.dremio.com/blog/the-evolution-of-apache-iceberg-catalogs/) +**Date**: May 24th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [From JSON, CSV and Parquet to Dashboards with Apache Iceberg and Dremio](https://www.dremio.com/blog/from-json-csv-and-parquet-to-dashboards-with-apache-iceberg-and-dremio/) +**Date**: May 13th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [From Apache Druid to Dashboards with Dremio and Apache Iceberg](https://www.dremio.com/blog/from-apache-druid-to-dashboards-with-dremio-and-apache-iceberg/) +**Date**: May 13th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Ingesting Data into Nessie & Apache Iceberg with kafka-connect and querying it with Dremio](https://www.dremio.com/blog/ingesting-data-into-nessie-apache-iceberg-with-kafka-connect-and-querying-it-with-dremio/) +**Date**: May 10th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [From MySQL to Dashboards with Dremio and Apache Iceberg](https://www.dremio.com/blog/from-mysql-to-dashboards-with-dremio-and-apache-iceberg/) +**Date**: May 7th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [From Elasticsearch to Dashboards with Dremio and Apache Iceberg](https://www.dremio.com/blog/from-elasticsearch-to-dashboards-with-dremio-and-apache-iceberg/) +**Date**: May 7th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Streaming and Batch Data Lakehouses with Apache Iceberg, Dremio and Upsolver](https://www.dremio.com/blog/streaming-and-batch-data-lakehouses-with-apache-iceberg-dremio-and-upsolver/) +**Date**: April 15th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [End-to-End Basic Data Engineering Tutorial (Apache Spark, Apache Iceberg, Dremio, Apache Superset, Nessie)](https://medium.com/data-engineering-with-dremio/end-to-end-basic-data-engineering-tutorial-spark-dremio-superset-c076a56eaa75) **Date**: April 1st, 2024, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) From 9e895cb6dff9dcd2a117a4f5e197f0235047ff54 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 30 Oct 2024 10:29:45 +0100 Subject: [PATCH 072/313] AWS: Refresh vended credentials (#11389) --- .../iceberg/aws/AwsClientProperties.java | 34 +- .../iceberg/aws/s3/S3FileIOProperties.java | 7 + .../aws/s3/VendedCredentialsProvider.java | 138 ++++++++ .../iceberg/aws/AwsClientPropertiesTest.java | 29 ++ .../aws/s3/TestVendedCredentialsProvider.java | 323 ++++++++++++++++++ 5 files changed, 526 insertions(+), 5 deletions(-) create mode 100644 aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java create mode 100644 aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.java diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java index 0c91f8685ae9..4f2d4d6a5a6c 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.Map; +import org.apache.iceberg.aws.s3.VendedCredentialsProvider; import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.common.DynMethods; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -66,14 +67,27 @@ public class AwsClientProperties implements Serializable { */ public static final String CLIENT_REGION = "client.region"; + /** + * When set, the {@link VendedCredentialsProvider} will be used to fetch and refresh vended + * credentials from this endpoint. + */ + public static final String REFRESH_CREDENTIALS_ENDPOINT = "client.refresh-credentials-endpoint"; + + /** Controls whether vended credentials should be refreshed or not. Defaults to true. */ + public static final String REFRESH_CREDENTIALS_ENABLED = "client.refresh-credentials-enabled"; + private String clientRegion; private final String clientCredentialsProvider; private final Map clientCredentialsProviderProperties; + private final String refreshCredentialsEndpoint; + private final boolean refreshCredentialsEnabled; public AwsClientProperties() { this.clientRegion = null; this.clientCredentialsProvider = null; this.clientCredentialsProviderProperties = null; + this.refreshCredentialsEndpoint = null; + this.refreshCredentialsEnabled = true; } public AwsClientProperties(Map properties) { @@ -81,6 +95,9 @@ public AwsClientProperties(Map properties) { this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER); this.clientCredentialsProviderProperties = PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX); + this.refreshCredentialsEndpoint = properties.get(REFRESH_CREDENTIALS_ENDPOINT); + this.refreshCredentialsEnabled = + PropertyUtil.propertyAsBoolean(properties, REFRESH_CREDENTIALS_ENABLED, true); } public String clientRegion() { @@ -122,11 +139,12 @@ public void applyClientCredentialConfigurations(T b } /** - * Returns a credentials provider instance. If params were set, we return a new credentials - * instance. If none of the params are set, we try to dynamically load the provided credentials - * provider class. Upon loading the class, we try to invoke {@code create(Map)} - * static method. If that fails, we fall back to {@code create()}. If credential provider class - * wasn't set, we fall back to default credentials provider. + * Returns a credentials provider instance. If {@link #refreshCredentialsEndpoint} is set, an + * instance of {@link VendedCredentialsProvider} is returned. If params were set, we return a new + * credentials instance. If none of the params are set, we try to dynamically load the provided + * credentials provider class. Upon loading the class, we try to invoke {@code create(Map)} static method. If that fails, we fall back to {@code create()}. If credential + * provider class wasn't set, we fall back to default credentials provider. * * @param accessKeyId the AWS access key ID * @param secretAccessKey the AWS secret access key @@ -136,6 +154,12 @@ public void applyClientCredentialConfigurations(T b @SuppressWarnings("checkstyle:HiddenField") public AwsCredentialsProvider credentialsProvider( String accessKeyId, String secretAccessKey, String sessionToken) { + if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { + clientCredentialsProviderProperties.put( + VendedCredentialsProvider.URI, refreshCredentialsEndpoint); + return credentialsProvider(VendedCredentialsProvider.class.getName()); + } + if (!Strings.isNullOrEmpty(accessKeyId) && !Strings.isNullOrEmpty(secretAccessKey)) { if (Strings.isNullOrEmpty(sessionToken)) { return StaticCredentialsProvider.create( diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java index 5da758704ae5..8d97b9d1bf20 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java @@ -225,6 +225,13 @@ public class S3FileIOProperties implements Serializable { */ public static final String SESSION_TOKEN = "s3.session-token"; + /** + * Configure the expiration time in millis of the static session token used to access S3FileIO. + * This expiration time is currently only used in {@link VendedCredentialsProvider} for refreshing + * vended credentials. + */ + static final String SESSION_TOKEN_EXPIRES_AT_MS = "s3.session-token-expires-at-ms"; + /** * Enable to make S3FileIO, to make cross-region call to the region specified in the ARN of an * access point. diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java b/aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java new file mode 100644 index 000000000000..e249d3ff1dec --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java @@ -0,0 +1,138 @@ +/* + * 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.aws.s3; + +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.ErrorHandlers; +import org.apache.iceberg.rest.HTTPClient; +import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.auth.OAuth2Properties; +import org.apache.iceberg.rest.auth.OAuth2Util; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.responses.LoadCredentialsResponse; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; +import software.amazon.awssdk.utils.IoUtils; +import software.amazon.awssdk.utils.SdkAutoCloseable; +import software.amazon.awssdk.utils.cache.CachedSupplier; +import software.amazon.awssdk.utils.cache.RefreshResult; + +public class VendedCredentialsProvider implements AwsCredentialsProvider, SdkAutoCloseable { + public static final String URI = "credentials.uri"; + private volatile HTTPClient client; + private final Map properties; + private final CachedSupplier credentialCache; + + private VendedCredentialsProvider(Map properties) { + Preconditions.checkArgument(null != properties, "Invalid properties: null"); + Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: null"); + this.properties = properties; + this.credentialCache = + CachedSupplier.builder(this::refreshCredential) + .cachedValueName(VendedCredentialsProvider.class.getName()) + .build(); + } + + @Override + public AwsCredentials resolveCredentials() { + return credentialCache.get(); + } + + @Override + public void close() { + IoUtils.closeQuietly(client, null); + credentialCache.close(); + } + + public static VendedCredentialsProvider create(Map properties) { + return new VendedCredentialsProvider(properties); + } + + private RESTClient httpClient() { + if (null == client) { + synchronized (this) { + if (null == client) { + client = HTTPClient.builder(properties).uri(properties.get(URI)).build(); + } + } + } + + return client; + } + + private LoadCredentialsResponse fetchCredentials() { + return httpClient() + .get( + properties.get(URI), + null, + LoadCredentialsResponse.class, + OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)), + ErrorHandlers.defaultErrorHandler()); + } + + private RefreshResult refreshCredential() { + LoadCredentialsResponse response = fetchCredentials(); + + List s3Credentials = + response.credentials().stream() + .filter(c -> c.prefix().startsWith("s3")) + .collect(Collectors.toList()); + + Preconditions.checkState(!s3Credentials.isEmpty(), "Invalid S3 Credentials: empty"); + Preconditions.checkState( + s3Credentials.size() == 1, "Invalid S3 Credentials: only one S3 credential should exist"); + + Credential s3Credential = s3Credentials.get(0); + checkCredential(s3Credential, S3FileIOProperties.ACCESS_KEY_ID); + checkCredential(s3Credential, S3FileIOProperties.SECRET_ACCESS_KEY); + checkCredential(s3Credential, S3FileIOProperties.SESSION_TOKEN); + checkCredential(s3Credential, S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS); + + String accessKeyId = s3Credential.config().get(S3FileIOProperties.ACCESS_KEY_ID); + String secretAccessKey = s3Credential.config().get(S3FileIOProperties.SECRET_ACCESS_KEY); + String sessionToken = s3Credential.config().get(S3FileIOProperties.SESSION_TOKEN); + String tokenExpiresAtMillis = + s3Credential.config().get(S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS); + Instant expiresAt = Instant.ofEpochMilli(Long.parseLong(tokenExpiresAtMillis)); + Instant prefetchAt = expiresAt.minus(5, ChronoUnit.MINUTES); + + return RefreshResult.builder( + (AwsCredentials) + AwsSessionCredentials.builder() + .accessKeyId(accessKeyId) + .secretAccessKey(secretAccessKey) + .sessionToken(sessionToken) + .expirationTime(expiresAt) + .build()) + .staleTime(expiresAt) + .prefetchTime(prefetchAt) + .build(); + } + + private void checkCredential(Credential credential, String property) { + Preconditions.checkState( + credential.config().containsKey(property), "Invalid S3 Credentials: %s not set", property); + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java b/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java index c318538d9509..5cf9dd810c9f 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java +++ b/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java @@ -21,6 +21,8 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; +import org.apache.iceberg.aws.s3.VendedCredentialsProvider; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; @@ -29,6 +31,7 @@ import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3ClientBuilder; @@ -111,4 +114,30 @@ public void testSessionCredentialsConfiguration() { .as("The secret access key should be the same as the one set by tag SECRET_ACCESS_KEY") .isEqualTo("secret"); } + + @Test + public void refreshCredentialsEndpoint() { + AwsClientProperties awsClientProperties = + new AwsClientProperties( + ImmutableMap.of( + AwsClientProperties.REFRESH_CREDENTIALS_ENDPOINT, + "http://localhost:1234/v1/credentials")); + + assertThat(awsClientProperties.credentialsProvider("key", "secret", "token")) + .isInstanceOf(VendedCredentialsProvider.class); + } + + @Test + public void refreshCredentialsEndpointSetButRefreshDisabled() { + AwsClientProperties awsClientProperties = + new AwsClientProperties( + ImmutableMap.of( + AwsClientProperties.REFRESH_CREDENTIALS_ENABLED, + "false", + AwsClientProperties.REFRESH_CREDENTIALS_ENDPOINT, + "http://localhost:1234/v1/credentials")); + + assertThat(awsClientProperties.credentialsProvider("key", "secret", "token")) + .isInstanceOf(StaticCredentialsProvider.class); + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.java new file mode 100644 index 000000000000..67cd1cb55241 --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.java @@ -0,0 +1,323 @@ +/* + * 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.aws.s3; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockserver.integration.ClientAndServer.startClientAndServer; +import static org.mockserver.model.HttpRequest.request; +import static org.mockserver.model.HttpResponse.response; + +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import org.apache.iceberg.exceptions.RESTException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.HttpMethod; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.ImmutableCredential; +import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponseParser; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockserver.integration.ClientAndServer; +import org.mockserver.model.HttpRequest; +import org.mockserver.model.HttpResponse; +import org.mockserver.verify.VerificationTimes; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; + +public class TestVendedCredentialsProvider { + + private static final int PORT = 3232; + private static final String URI = String.format("http://127.0.0.1:%d/v1/credentials", PORT); + private static ClientAndServer mockServer; + + @BeforeAll + public static void beforeAll() { + mockServer = startClientAndServer(PORT); + } + + @AfterAll + public static void stopServer() { + mockServer.stop(); + } + + @BeforeEach + public void before() { + mockServer.reset(); + } + + @Test + public void invalidOrMissingUri() { + assertThatThrownBy(() -> VendedCredentialsProvider.create(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid properties: null"); + assertThatThrownBy(() -> VendedCredentialsProvider.create(ImmutableMap.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid URI: null"); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create( + ImmutableMap.of(VendedCredentialsProvider.URI, "invalid uri"))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(RESTException.class) + .hasMessageStartingWith("Failed to create request URI from base invalid uri"); + } + } + + @Test + public void noS3Credentials() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + + HttpResponse mockResponse = + response( + LoadCredentialsResponseParser.toJson( + ImmutableLoadCredentialsResponse.builder().build())) + .withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid S3 Credentials: empty"); + } + } + + @Test + public void accessKeyIdAndSecretAccessKeyWithoutToken() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder() + .addCredentials( + ImmutableCredential.builder() + .prefix("s3") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey")) + .build()) + .build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid S3 Credentials: s3.session-token not set"); + } + } + + @Test + public void expirationNotSet() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder() + .addCredentials( + ImmutableCredential.builder() + .prefix("s3") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken")) + .build()) + .build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid S3 Credentials: s3.session-token-expires-at-ms not set"); + } + } + + @Test + public void nonExpiredToken() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credential = + ImmutableCredential.builder() + .prefix("s3") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + AwsCredentials awsCredentials = provider.resolveCredentials(); + + verifyCredentials(awsCredentials, credential); + + for (int i = 0; i < 5; i++) { + // resolving credentials multiple times should not hit the credentials endpoint again + assertThat(provider.resolveCredentials()).isSameAs(awsCredentials); + } + } + + mockServer.verify(mockRequest, VerificationTimes.once()); + } + + @Test + public void expiredToken() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credential = + ImmutableCredential.builder() + .prefix("s3") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().minus(1, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + AwsCredentials awsCredentials = provider.resolveCredentials(); + verifyCredentials(awsCredentials, credential); + + // resolving credentials multiple times should hit the credentials endpoint again + AwsCredentials refreshedCredentials = provider.resolveCredentials(); + assertThat(refreshedCredentials).isNotSameAs(awsCredentials); + verifyCredentials(refreshedCredentials, credential); + } + + mockServer.verify(mockRequest, VerificationTimes.exactly(2)); + } + + @Test + public void multipleS3Credentials() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credentialOne = + ImmutableCredential.builder() + .prefix("gcs") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey1", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey1", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken1", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + Credential credentialTwo = + ImmutableCredential.builder() + .prefix("s3://custom-uri/longest-prefix") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey2", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey2", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken2", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().plus(2, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + Credential credentialThree = + ImmutableCredential.builder() + .prefix("s3://custom-uri/long") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey3", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey3", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken3", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().plus(3, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder() + .addCredentials(credentialOne, credentialTwo, credentialThree) + .build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedCredentialsProvider provider = + VendedCredentialsProvider.create(ImmutableMap.of(VendedCredentialsProvider.URI, URI))) { + assertThatThrownBy(provider::resolveCredentials) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid S3 Credentials: only one S3 credential should exist"); + } + } + + private void verifyCredentials(AwsCredentials awsCredentials, Credential credential) { + assertThat(awsCredentials).isInstanceOf(AwsSessionCredentials.class); + AwsSessionCredentials creds = (AwsSessionCredentials) awsCredentials; + + assertThat(creds.accessKeyId()) + .isEqualTo(credential.config().get(S3FileIOProperties.ACCESS_KEY_ID)); + assertThat(creds.secretAccessKey()) + .isEqualTo(credential.config().get(S3FileIOProperties.SECRET_ACCESS_KEY)); + assertThat(creds.sessionToken()) + .isEqualTo(credential.config().get(S3FileIOProperties.SESSION_TOKEN)); + assertThat(creds.expirationTime()) + .isPresent() + .get() + .extracting(Instant::toEpochMilli) + .isEqualTo( + Long.parseLong( + credential.config().get(S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS))); + } +} From 7c390861935874d999aad66ebafd4ef9aba648d9 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 30 Oct 2024 15:49:37 +0100 Subject: [PATCH 073/313] Build: Bump Hadoop to 3.4.1 (#11428) --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 13ce4da5471b..066c3549769f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -47,7 +47,7 @@ flink120 = { strictly = "1.20.0"} google-libraries-bom = "26.49.0" guava = "33.3.1-jre" hadoop2 = "2.7.3" -hadoop3 = "3.3.6" +hadoop3 = "3.4.1" httpcomponents-httpclient5 = "5.4" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" From dec84c0549e80b113b3bb6799dcaf3f6d3c59fed Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 30 Oct 2024 18:30:55 +0100 Subject: [PATCH 074/313] Core: Remove credentials from LoadViewResponse (#11432) --- .../rest/responses/LoadViewResponse.java | 8 -- .../responses/LoadViewResponseParser.java | 16 --- .../responses/TestLoadViewResponseParser.java | 111 ------------------ 3 files changed, 135 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java index d7f9040e77f7..d07ba872fdaa 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java @@ -18,11 +18,8 @@ */ package org.apache.iceberg.rest.responses; -import java.util.List; import java.util.Map; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.rest.RESTResponse; -import org.apache.iceberg.rest.credentials.Credential; import org.apache.iceberg.view.ViewMetadata; import org.immutables.value.Value; @@ -34,11 +31,6 @@ public interface LoadViewResponse extends RESTResponse { Map config(); - @Value.Default - default List credentials() { - return ImmutableList.of(); - } - @Override default void validate() { // nothing to validate as it's not possible to create an invalid instance diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java index aedf05cf62a9..a8aaf17e5d76 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java @@ -22,8 +22,6 @@ import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.rest.credentials.Credential; -import org.apache.iceberg.rest.credentials.CredentialParser; import org.apache.iceberg.util.JsonUtil; import org.apache.iceberg.view.ViewMetadata; import org.apache.iceberg.view.ViewMetadataParser; @@ -33,7 +31,6 @@ public class LoadViewResponseParser { private static final String METADATA_LOCATION = "metadata-location"; private static final String METADATA = "metadata"; private static final String CONFIG = "config"; - private static final String STORAGE_CREDENTIALS = "storage-credentials"; private LoadViewResponseParser() {} @@ -59,15 +56,6 @@ public static void toJson(LoadViewResponse response, JsonGenerator gen) throws I JsonUtil.writeStringMap(CONFIG, response.config(), gen); } - if (!response.credentials().isEmpty()) { - gen.writeArrayFieldStart(STORAGE_CREDENTIALS); - for (Credential credential : response.credentials()) { - CredentialParser.toJson(credential, gen); - } - - gen.writeEndArray(); - } - gen.writeEndObject(); } @@ -92,10 +80,6 @@ public static LoadViewResponse fromJson(JsonNode json) { builder.config(JsonUtil.getStringMap(CONFIG, json)); } - if (json.hasNonNull(STORAGE_CREDENTIALS)) { - builder.addAllCredentials(LoadCredentialsResponseParser.fromJson(json).credentials()); - } - return builder.build(); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java index 086db0fec8b4..f3de08cd2912 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java @@ -25,7 +25,6 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.rest.credentials.ImmutableCredential; import org.apache.iceberg.types.Types; import org.apache.iceberg.view.ImmutableViewVersion; import org.apache.iceberg.view.ViewMetadata; @@ -246,114 +245,4 @@ public void roundTripSerdeWithConfig() { assertThat(LoadViewResponseParser.toJson(LoadViewResponseParser.fromJson(json), true)) .isEqualTo(expectedJson); } - - @Test - public void roundTripSerdeWithCredentials() { - String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; - ViewMetadata viewMetadata = - ViewMetadata.builder() - .assignUUID(uuid) - .setLocation("location") - .addSchema(new Schema(Types.NestedField.required(1, "x", Types.LongType.get()))) - .addVersion( - ImmutableViewVersion.builder() - .schemaId(0) - .versionId(1) - .timestampMillis(23L) - .defaultNamespace(Namespace.of("ns1")) - .build()) - .setCurrentVersionId(1) - .build(); - - LoadViewResponse response = - ImmutableLoadViewResponse.builder() - .metadata(viewMetadata) - .metadataLocation("custom-location") - .addCredentials( - ImmutableCredential.builder() - .prefix("s3://custom-uri") - .config( - ImmutableMap.of( - "s3.access-key-id", - "keyId", - "s3.secret-access-key", - "accessKey", - "s3.session-token", - "sessionToken")) - .build()) - .addCredentials( - ImmutableCredential.builder() - .prefix("gs://custom-uri") - .config( - ImmutableMap.of( - "gcs.oauth2.token", "gcsToken1", "gcs.oauth2.token-expires-at", "1000")) - .build()) - .addCredentials( - ImmutableCredential.builder() - .prefix("gs") - .config( - ImmutableMap.of( - "gcs.oauth2.token", "gcsToken2", "gcs.oauth2.token-expires-at", "2000")) - .build()) - .build(); - - String expectedJson = - "{\n" - + " \"metadata-location\" : \"custom-location\",\n" - + " \"metadata\" : {\n" - + " \"view-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" - + " \"format-version\" : 1,\n" - + " \"location\" : \"location\",\n" - + " \"schemas\" : [ {\n" - + " \"type\" : \"struct\",\n" - + " \"schema-id\" : 0,\n" - + " \"fields\" : [ {\n" - + " \"id\" : 1,\n" - + " \"name\" : \"x\",\n" - + " \"required\" : true,\n" - + " \"type\" : \"long\"\n" - + " } ]\n" - + " } ],\n" - + " \"current-version-id\" : 1,\n" - + " \"versions\" : [ {\n" - + " \"version-id\" : 1,\n" - + " \"timestamp-ms\" : 23,\n" - + " \"schema-id\" : 0,\n" - + " \"summary\" : { },\n" - + " \"default-namespace\" : [ \"ns1\" ],\n" - + " \"representations\" : [ ]\n" - + " } ],\n" - + " \"version-log\" : [ {\n" - + " \"timestamp-ms\" : 23,\n" - + " \"version-id\" : 1\n" - + " } ]\n" - + " },\n" - + " \"storage-credentials\" : [ {\n" - + " \"prefix\" : \"s3://custom-uri\",\n" - + " \"config\" : {\n" - + " \"s3.access-key-id\" : \"keyId\",\n" - + " \"s3.secret-access-key\" : \"accessKey\",\n" - + " \"s3.session-token\" : \"sessionToken\"\n" - + " }\n" - + " }, {\n" - + " \"prefix\" : \"gs://custom-uri\",\n" - + " \"config\" : {\n" - + " \"gcs.oauth2.token\" : \"gcsToken1\",\n" - + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" - + " }\n" - + " }, {\n" - + " \"prefix\" : \"gs\",\n" - + " \"config\" : {\n" - + " \"gcs.oauth2.token\" : \"gcsToken2\",\n" - + " \"gcs.oauth2.token-expires-at\" : \"2000\"\n" - + " }\n" - + " } ]\n" - + "}"; - - String json = LoadViewResponseParser.toJson(response, true); - assertThat(json).isEqualTo(expectedJson); - // can't do an equality comparison because Schema doesn't implement equals/hashCode - assertThat(LoadViewResponseParser.toJson(LoadViewResponseParser.fromJson(json), true)) - .isEqualTo(expectedJson); - } } From f4b36a5e553cc8a14208890da2a3adc91ceefbda Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 30 Oct 2024 18:52:34 +0100 Subject: [PATCH 075/313] OpenAPI: Remove credentials from LoadViewResult (#11433) --- open-api/rest-catalog-open-api.py | 8 -------- open-api/rest-catalog-open-api.yaml | 9 --------- 2 files changed, 17 deletions(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index e9cce361dd88..684e4bdb0fa6 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -1333,19 +1333,11 @@ class LoadViewResult(BaseModel): - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled - ## Storage Credentials - - Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. - Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. - """ metadata_location: str = Field(..., alias='metadata-location') metadata: ViewMetadata config: Optional[Dict[str, str]] = None - storage_credentials: Optional[List[StorageCredential]] = Field( - None, alias='storage-credentials' - ) class ReportMetricsRequest(BaseModel): diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 9835a96e0aa3..d91e32ec493e 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -3466,11 +3466,6 @@ components: ## General Configurations - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled - - ## Storage Credentials - - Credentials for ADLS / GCS / S3 / ... are provided through the `storage-credentials` field. - Clients must first check whether the respective credentials exist in the `storage-credentials` field before checking the `config` for credentials. type: object required: - metadata-location @@ -3484,10 +3479,6 @@ components: type: object additionalProperties: type: string - storage-credentials: - type: array - items: - $ref: '#/components/schemas/StorageCredential' TokenType: type: string From 91e04c9c88b63dc01d6c8e69dfdc8cd27ee811cc Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 30 Oct 2024 14:58:25 -0700 Subject: [PATCH 076/313] API: Add compatibility checks for Schemas with default values (#11434) Co-authored-by: Russell Spitzer Co-authored-by: Fokko Driesprong --- .../main/java/org/apache/iceberg/Schema.java | 38 ++++-- .../java/org/apache/iceberg/TestSchema.java | 111 ++++++++++++++++++ .../org/apache/iceberg/TestTableMetadata.java | 3 +- 3 files changed, 143 insertions(+), 9 deletions(-) create mode 100644 api/src/test/java/org/apache/iceberg/TestSchema.java diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index 9bcf691f5a03..44f65ff56a54 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -54,6 +54,7 @@ public class Schema implements Serializable { private static final Joiner NEWLINE = Joiner.on('\n'); private static final String ALL_COLUMNS = "*"; private static final int DEFAULT_SCHEMA_ID = 0; + private static final int DEFAULT_VALUES_MIN_FORMAT_VERSION = 3; private static final Map MIN_FORMAT_VERSIONS = ImmutableMap.of(Type.TypeID.TIMESTAMP_NANO, 3); @@ -586,16 +587,37 @@ private List reassignIds(List columns, TypeUtil.GetID * @param formatVersion table format version */ public static void checkCompatibility(Schema schema, int formatVersion) { - // check the type in each field + // accumulate errors as a treemap to keep them in a reasonable order + Map problems = Maps.newTreeMap(); + + // check each field's type and defaults for (NestedField field : schema.lazyIdToField().values()) { Integer minFormatVersion = MIN_FORMAT_VERSIONS.get(field.type().typeId()); - Preconditions.checkState( - minFormatVersion == null || formatVersion >= minFormatVersion, - "Invalid type in v%s schema: %s %s is not supported until v%s", - formatVersion, - schema.findColumnName(field.fieldId()), - field.type(), - minFormatVersion); + if (minFormatVersion != null && formatVersion < minFormatVersion) { + problems.put( + field.fieldId(), + String.format( + "Invalid type for %s: %s is not supported until v%s", + schema.findColumnName(field.fieldId()), field.type(), minFormatVersion)); + } + + if (field.initialDefault() != null && formatVersion < DEFAULT_VALUES_MIN_FORMAT_VERSION) { + problems.put( + field.fieldId(), + String.format( + "Invalid initial default for %s: non-null default (%s) is not supported until v%s", + schema.findColumnName(field.fieldId()), + field.initialDefault(), + DEFAULT_VALUES_MIN_FORMAT_VERSION)); + } + } + + // throw if there are any compatibility problems + if (!problems.isEmpty()) { + throw new IllegalStateException( + String.format( + "Invalid schema for v%s:\n- %s", + formatVersion, Joiner.on("\n- ").join(problems.values()))); } } } diff --git a/api/src/test/java/org/apache/iceberg/TestSchema.java b/api/src/test/java/org/apache/iceberg/TestSchema.java new file mode 100644 index 000000000000..fec7343c5cbc --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/TestSchema.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestSchema { + private static final Schema TS_NANO_CASES = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "ts", Types.TimestampNanoType.withZone()), + Types.NestedField.optional( + 3, "arr", Types.ListType.ofRequired(4, Types.TimestampNanoType.withoutZone())), + Types.NestedField.required( + 5, + "struct", + Types.StructType.of( + Types.NestedField.optional(6, "inner_ts", Types.TimestampNanoType.withZone()), + Types.NestedField.required(7, "data", Types.StringType.get()))), + Types.NestedField.optional( + 8, + "struct_arr", + Types.StructType.of( + Types.NestedField.optional(9, "ts", Types.TimestampNanoType.withoutZone())))); + + private static final Schema INITIAL_DEFAULT_SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required("has_default") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("--") + .withWriteDefault("--") + .build()); + + private static final Schema WRITE_DEFAULT_SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required("has_default") + .withId(2) + .ofType(Types.StringType.get()) + .withWriteDefault("--") + .build()); + + @ParameterizedTest + @ValueSource(ints = {1, 2}) + public void testUnsupportedTimestampNano(int formatVersion) { + assertThatThrownBy(() -> Schema.checkCompatibility(TS_NANO_CASES, formatVersion)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid schema for v%s:\n" + + "- Invalid type for ts: timestamptz_ns is not supported until v3\n" + + "- Invalid type for arr.element: timestamp_ns is not supported until v3\n" + + "- Invalid type for struct.inner_ts: timestamptz_ns is not supported until v3\n" + + "- Invalid type for struct_arr.ts: timestamp_ns is not supported until v3", + formatVersion); + } + + @Test + public void testSupportedTimestampNano() { + assertThatCode(() -> Schema.checkCompatibility(TS_NANO_CASES, 3)).doesNotThrowAnyException(); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2}) + public void testUnsupportedInitialDefault(int formatVersion) { + assertThatThrownBy(() -> Schema.checkCompatibility(INITIAL_DEFAULT_SCHEMA, formatVersion)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid schema for v%s:\n" + + "- Invalid initial default for has_default: " + + "non-null default (--) is not supported until v3", + formatVersion); + } + + @Test + public void testSupportedInitialDefault() { + assertThatCode(() -> Schema.checkCompatibility(INITIAL_DEFAULT_SCHEMA, 3)) + .doesNotThrowAnyException(); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 3}) + public void testSupportedWriteDefault(int formatVersion) { + // only the initial default is a forward-incompatible change + assertThatCode(() -> Schema.checkCompatibility(WRITE_DEFAULT_SCHEMA, formatVersion)) + .doesNotThrowAnyException(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 5ada35765773..71254b3abb1b 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -1672,7 +1672,8 @@ public void testV3TimestampNanoTypeSupport() { unsupportedFormatVersion)) .isInstanceOf(IllegalStateException.class) .hasMessage( - "Invalid type in v%s schema: struct.ts_nanos timestamptz_ns is not supported until v3", + "Invalid schema for v%s:\n" + + "- Invalid type for struct.ts_nanos: timestamptz_ns is not supported until v3", unsupportedFormatVersion); } From 57fb6d56588ea91e995663b1a6f8bfee34060fa8 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Thu, 31 Oct 2024 13:31:14 -0700 Subject: [PATCH 077/313] Doc: Update rewrite data files spark procedure (#11396) --- docs/docs/spark-procedures.md | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index 0953e729a77b..40407b8c0f84 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -393,6 +393,7 @@ Iceberg can compact data files in parallel using Spark with the `rewriteDataFile | `max-concurrent-file-group-rewrites` | 5 | Maximum number of file groups to be simultaneously rewritten | | `partial-progress.enabled` | false | Enable committing groups of files prior to the entire rewrite completing | | `partial-progress.max-commits` | 10 | Maximum amount of commits that this rewrite is allowed to produce if partial progress is enabled | +| `partial-progress.max-failed-commits` | value of `partital-progress.max-commits` | Maximum amount of failed commits allowed before job failure, if partial progress is enabled | | `use-starting-sequence-number` | true | Use the sequence number of the snapshot at compaction start time instead of that of the newly produced snapshot | | `rewrite-job-order` | none | Force the rewrite job order based on the value.
  • If rewrite-job-order=bytes-asc, then rewrite the smallest job groups first.
  • If rewrite-job-order=bytes-desc, then rewrite the largest job groups first.
  • If rewrite-job-order=files-asc, then rewrite the job groups with the least files first.
  • If rewrite-job-order=files-desc, then rewrite the job groups with the most files first.
  • If rewrite-job-order=none, then rewrite job groups in the order they were planned (no specific ordering).
| | `target-file-size-bytes` | 536870912 (512 MB, default value of `write.target-file-size-bytes` from [table properties](configuration.md#write-properties)) | Target output file size | @@ -402,7 +403,13 @@ Iceberg can compact data files in parallel using Spark with the `rewriteDataFile | `rewrite-all` | false | Force rewriting of all provided files overriding other options | | `max-file-group-size-bytes` | 107374182400 (100GB) | Largest amount of data that should be rewritten in a single file group. The entire rewrite operation is broken down into pieces based on partitioning and within partitions based on size into file-groups. This helps with breaking down the rewriting of very large partitions which may not be rewritable otherwise due to the resource constraints of the cluster. | | `delete-file-threshold` | 2147483647 | Minimum number of deletes that needs to be associated with a data file for it to be considered for rewriting | +| `output-spec-id` | current partition spec id | Identifier of the output partition spec. Data will be reorganized during the rewrite to align with the output partitioning. | +| `remove-dangling-deletes` | false | Remove dangling position and equality deletes after rewriting. A delete file is considered dangling if it does not apply to any live data files. Enabling this will generate an additional commit for the removal. | +!!! info + Dangling delete files are removed based solely on data sequence numbers. This action does not apply to global + equality deletes or invalid equality deletes if their delete conditions do not match any data files, + nor to position delete files containing position deletes no longer matching any live data files. ##### Options for sort strategy @@ -447,9 +454,9 @@ Using the same defaults as bin-pack to determine which files to rewrite. CALL catalog_name.system.rewrite_data_files(table => 'db.sample', strategy => 'sort', sort_order => 'zorder(c1,c2)'); ``` -Rewrite the data files in table `db.sample` using bin-pack strategy in any partition where more than 2 or more files need to be rewritten. +Rewrite the data files in table `db.sample` using bin-pack strategy in any partition where at least two files need rewriting, and then remove any dangling delete files. ```sql -CALL catalog_name.system.rewrite_data_files(table => 'db.sample', options => map('min-input-files','2')); +CALL catalog_name.system.rewrite_data_files(table => 'db.sample', options => map('min-input-files', '2', 'remove-dangling-deletes', 'true')); ``` Rewrite the data files in table `db.sample` and select the files that may contain data matching the filter (id = 3 and name = "foo") to be rewritten. From ea61ee46db17d94f22a5ef11fd913146557bdce7 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 1 Nov 2024 04:48:45 +0800 Subject: [PATCH 078/313] Docs: warn `parallelism > 1` doesn't work for migration procedures (#11417) --- docs/docs/spark-procedures.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index 40407b8c0f84..3dc3926b92e4 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -593,6 +593,9 @@ See [`migrate`](#migrate) to replace an existing table with an Iceberg table. | `properties` | ️ | map | Properties to add to the newly created table | | `parallelism` | | int | Number of threads to use for file reading (defaults to 1) | +!!! warning + There's a [known issue with `parallelism > 1`](https://github.com/apache/iceberg/issues/11147) that is scheduled to be fixed in the next release. + #### Output | Output Name | Type | Description | @@ -636,6 +639,9 @@ By default, the original table is retained with the name `table_BACKUP_`. | `backup_table_name` | | string | Name of the table that will be retained as backup (defaults to `table_BACKUP_`) | | `parallelism` | | int | Number of threads to use for file reading (defaults to 1) | +!!! warning + There's a [known issue with `parallelism > 1`](https://github.com/apache/iceberg/issues/11147) that is scheduled to be fixed in the next release. + #### Output | Output Name | Type | Description | @@ -682,6 +688,9 @@ Warning : Schema is not validated, adding files with different schema to the Ice Warning : Files added by this method can be physically deleted by Iceberg operations +!!! warning + There's a [known issue with `parallelism > 1`](https://github.com/apache/iceberg/issues/11147) that is scheduled to be fixed in the next release. + #### Output | Output Name | Type | Description | From 1d4df34d7bd06f6f69856aec15a1a94ae104c605 Mon Sep 17 00:00:00 2001 From: sullis Date: Thu, 31 Oct 2024 14:33:14 -0700 Subject: [PATCH 079/313] Core: Log retry sleep time (#11413) --- core/src/main/java/org/apache/iceberg/util/Tasks.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/Tasks.java b/core/src/main/java/org/apache/iceberg/util/Tasks.java index 14804e040755..29100c6cffb2 100644 --- a/core/src/main/java/org/apache/iceberg/util/Tasks.java +++ b/core/src/main/java/org/apache/iceberg/util/Tasks.java @@ -454,11 +454,13 @@ private void runTaskWithRetry(Task task, I item) thr Math.min( minSleepTimeMs * Math.pow(scaleFactor, attempt - 1), (double) maxSleepTimeMs); int jitter = ThreadLocalRandom.current().nextInt(Math.max(1, (int) (delayMs * 0.1))); + int sleepTimeMs = delayMs + jitter; - LOG.warn("Retrying task after failure: {}", e.getMessage(), e); + LOG.warn( + "Retrying task after failure: sleepTimeMs={} {}", sleepTimeMs, e.getMessage(), e); try { - TimeUnit.MILLISECONDS.sleep(delayMs + jitter); + TimeUnit.MILLISECONDS.sleep(sleepTimeMs); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new RuntimeException(ie); From caf424a373fa125d427401acda7079b08abea9de Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 1 Nov 2024 20:18:34 +0100 Subject: [PATCH 080/313] Core: Use RoaringPositionBitmap in position index (#11441) --- .../deletes/BitmapPositionDeleteIndex.java | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java index a1b57a38666d..77ad7b47ac79 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java @@ -23,40 +23,39 @@ import java.util.function.LongConsumer; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.roaringbitmap.longlong.Roaring64Bitmap; class BitmapPositionDeleteIndex implements PositionDeleteIndex { - private final Roaring64Bitmap roaring64Bitmap; + private final RoaringPositionBitmap bitmap; private final List deleteFiles; BitmapPositionDeleteIndex() { - this.roaring64Bitmap = new Roaring64Bitmap(); + this.bitmap = new RoaringPositionBitmap(); this.deleteFiles = Lists.newArrayList(); } BitmapPositionDeleteIndex(Collection deleteFiles) { - this.roaring64Bitmap = new Roaring64Bitmap(); + this.bitmap = new RoaringPositionBitmap(); this.deleteFiles = Lists.newArrayList(deleteFiles); } BitmapPositionDeleteIndex(DeleteFile deleteFile) { - this.roaring64Bitmap = new Roaring64Bitmap(); + this.bitmap = new RoaringPositionBitmap(); this.deleteFiles = deleteFile != null ? Lists.newArrayList(deleteFile) : Lists.newArrayList(); } void merge(BitmapPositionDeleteIndex that) { - roaring64Bitmap.or(that.roaring64Bitmap); + bitmap.setAll(that.bitmap); deleteFiles.addAll(that.deleteFiles); } @Override public void delete(long position) { - roaring64Bitmap.add(position); + bitmap.set(position); } @Override public void delete(long posStart, long posEnd) { - roaring64Bitmap.addRange(posStart, posEnd); + bitmap.setRange(posStart, posEnd); } @Override @@ -71,17 +70,17 @@ public void merge(PositionDeleteIndex that) { @Override public boolean isDeleted(long position) { - return roaring64Bitmap.contains(position); + return bitmap.contains(position); } @Override public boolean isEmpty() { - return roaring64Bitmap.isEmpty(); + return bitmap.isEmpty(); } @Override public void forEach(LongConsumer consumer) { - roaring64Bitmap.forEach(consumer::accept); + bitmap.forEach(consumer); } @Override From fe23584fc3af9f0ea1371989030b0a99affb233f Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Fri, 1 Nov 2024 17:22:44 -0700 Subject: [PATCH 081/313] Core: Add validation for table commit properties (#11437) --- .../org/apache/iceberg/PropertiesUpdate.java | 9 ++-- .../org/apache/iceberg/TableMetadata.java | 6 +++ .../org/apache/iceberg/util/PropertyUtil.java | 50 +++++++++++++++++++ .../org/apache/iceberg/TestTransaction.java | 18 +++++++ .../iceberg/spark/sql/TestCreateTable.java | 42 ++++++++++++++++ 5 files changed, 121 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java index 35338a689205..9389aec50c0a 100644 --- a/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java +++ b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java @@ -98,12 +98,13 @@ public Map apply() { @Override public void commit() { + // If existing table commit properties in base are corrupted, allow rectification Tasks.foreach(ops) - .retry(base.propertyAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) + .retry(base.propertyTryAsInt(COMMIT_NUM_RETRIES, COMMIT_NUM_RETRIES_DEFAULT)) .exponentialBackoff( - base.propertyAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), - base.propertyAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), - base.propertyAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), + base.propertyTryAsInt(COMMIT_MIN_RETRY_WAIT_MS, COMMIT_MIN_RETRY_WAIT_MS_DEFAULT), + base.propertyTryAsInt(COMMIT_MAX_RETRY_WAIT_MS, COMMIT_MAX_RETRY_WAIT_MS_DEFAULT), + base.propertyTryAsInt(COMMIT_TOTAL_RETRY_TIME_MS, COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT), 2.0 /* exponential */) .onlyRetryOn(CommitFailedException.class) .run( diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index d20dd59d2b97..3cdc53995dce 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -134,6 +134,8 @@ static TableMetadata newTableMetadata( // break existing tables. MetricsConfig.fromProperties(properties).validateReferencedColumns(schema); + PropertyUtil.validateCommitProperties(properties); + return new Builder() .setInitialFormatVersion(formatVersion) .setCurrentSchema(freshSchema, lastColumnId.get()) @@ -486,6 +488,10 @@ public int propertyAsInt(String property, int defaultValue) { return PropertyUtil.propertyAsInt(properties, property, defaultValue); } + public int propertyTryAsInt(String property, int defaultValue) { + return PropertyUtil.propertyTryAsInt(properties, property, defaultValue); + } + public long propertyAsLong(String property, long defaultValue) { return PropertyUtil.propertyAsLong(properties, property, defaultValue); } diff --git a/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java b/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java index 68c8f3e9efda..633b0a6ae739 100644 --- a/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java @@ -24,10 +24,23 @@ import java.util.Set; import java.util.function.Predicate; import java.util.stream.Collectors; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class PropertyUtil { + private static final Logger LOG = LoggerFactory.getLogger(PropertyUtil.class); + + private static final Set COMMIT_PROPERTIES = + ImmutableSet.of( + TableProperties.COMMIT_NUM_RETRIES, + TableProperties.COMMIT_MIN_RETRY_WAIT_MS, + TableProperties.COMMIT_MAX_RETRY_WAIT_MS, + TableProperties.COMMIT_TOTAL_RETRY_TIME_MS); private PropertyUtil() {} @@ -57,6 +70,20 @@ public static double propertyAsDouble( return defaultValue; } + public static int propertyTryAsInt( + Map properties, String property, int defaultValue) { + String value = properties.get(property); + if (value == null) { + return defaultValue; + } + try { + return Integer.parseInt(value); + } catch (NumberFormatException e) { + LOG.warn("Failed to parse value of {} as integer, default to {}", property, defaultValue, e); + return defaultValue; + } + } + public static int propertyAsInt( Map properties, String property, int defaultValue) { String value = properties.get(property); @@ -100,6 +127,29 @@ public static String propertyAsString( return defaultValue; } + /** + * Validate the table commit related properties to have non-negative integer on table creation to + * prevent commit failure + */ + public static void validateCommitProperties(Map properties) { + for (String commitProperty : COMMIT_PROPERTIES) { + String value = properties.get(commitProperty); + if (value != null) { + int parsedValue; + try { + parsedValue = Integer.parseInt(value); + } catch (NumberFormatException e) { + throw new ValidationException( + "Table property %s must have integer value", commitProperty); + } + ValidationException.check( + parsedValue >= 0, + "Table property %s must have non negative integer value", + commitProperty); + } + } + } + /** * Returns subset of provided map with keys matching the provided prefix. Matching is * case-sensitive and the matching prefix is removed from the keys in returned map. diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 8fed7134fae1..8770e24f8e40 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -714,4 +714,22 @@ public void testTransactionRecommit() { assertThat(paths).isEqualTo(expectedPaths); assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); } + + @TestTemplate + public void testCommitProperties() { + table + .updateProperties() + .set(TableProperties.COMMIT_MAX_RETRY_WAIT_MS, "foo") + .set(TableProperties.COMMIT_NUM_RETRIES, "bar") + .set(TableProperties.COMMIT_TOTAL_RETRY_TIME_MS, Integer.toString(60 * 60 * 1000)) + .commit(); + table.updateProperties().remove(TableProperties.COMMIT_MAX_RETRY_WAIT_MS).commit(); + table.updateProperties().remove(TableProperties.COMMIT_NUM_RETRIES).commit(); + + assertThat(table.properties()) + .doesNotContainKey(TableProperties.COMMIT_NUM_RETRIES) + .doesNotContainKey(TableProperties.COMMIT_MAX_RETRY_WAIT_MS) + .containsEntry( + TableProperties.COMMIT_TOTAL_RETRY_TIME_MS, Integer.toString(60 * 60 * 1000)); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index ae0aa2cda49b..11d4cfebfea6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -31,6 +31,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.types.Types; @@ -348,6 +349,47 @@ public void testCreateTableProperties() { assertThat(table.properties()).containsEntry("p1", "2").containsEntry("p2", "x"); } + @TestTemplate + public void testCreateTableCommitProperties() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + assertThatThrownBy( + () -> + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('commit.retry.num-retries'='x', p2='x')", + tableName)) + .isInstanceOf(ValidationException.class) + .hasMessage("Table property commit.retry.num-retries must have integer value"); + + assertThatThrownBy( + () -> + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('commit.retry.max-wait-ms'='-1')", + tableName)) + .isInstanceOf(ValidationException.class) + .hasMessage("Table property commit.retry.max-wait-ms must have non negative integer value"); + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('commit.retry.num-retries'='1', 'commit.retry.max-wait-ms'='3000')", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.properties()) + .containsEntry(TableProperties.COMMIT_NUM_RETRIES, "1") + .containsEntry(TableProperties.COMMIT_MAX_RETRY_WAIT_MS, "3000"); + } + @TestTemplate public void testCreateTableWithFormatV2ThroughTableProperty() { assertThat(validationCatalog.tableExists(tableIdent)) From 8b4ebc66e1d34dc345f8370c341720cbda98f877 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Sat, 2 Nov 2024 10:06:12 +0100 Subject: [PATCH 082/313] Core: Add cardinality to PositionDeleteIndex (#11442) --- .../apache/iceberg/deletes/BitmapPositionDeleteIndex.java | 5 +++++ .../java/org/apache/iceberg/deletes/PositionDeleteIndex.java | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java index 77ad7b47ac79..cfb163e8379c 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java @@ -87,4 +87,9 @@ public void forEach(LongConsumer consumer) { public Collection deleteFiles() { return deleteFiles; } + + @Override + public long cardinality() { + return bitmap.cardinality(); + } } diff --git a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java index 3655b8b7e8eb..8ccfc03d1a26 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java @@ -87,6 +87,11 @@ default Collection deleteFiles() { return ImmutableList.of(); } + /** Returns the cardinality of this index. */ + default long cardinality() { + throw new UnsupportedOperationException(getClass().getName() + " does not support cardinality"); + } + /** Returns an empty immutable position delete index. */ static PositionDeleteIndex empty() { return EmptyPositionDeleteIndex.get(); From b9ebc71fbc9803b6a8a4b9ed63b9ad4adeb66edf Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sat, 2 Nov 2024 03:04:03 -0700 Subject: [PATCH 083/313] Puffin: Add deletion-vector-v1 blob type (#11238) --- format/puffin-spec.md | 55 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 55 insertions(+) diff --git a/format/puffin-spec.md b/format/puffin-spec.md index 7b4e3e6d96b3..0148db72e276 100644 --- a/format/puffin-spec.md +++ b/format/puffin-spec.md @@ -125,6 +125,61 @@ The blob metadata for this blob may include following properties: stored as non-negative integer value represented using decimal digits with no leading or trailing spaces. +#### `deletion-vector-v1` blob type + +A serialized delete vector (bitmap) that represents the positions of rows in a +file that are deleted. A set bit at position P indicates that the row at +position P is deleted. + +The vector supports positive 64-bit positions (the most significant bit must be +0), but is optimized for cases where most positions fit in 32 bits by using a +collection of 32-bit Roaring bitmaps. 64-bit positions are divided into a +32-bit "key" using the most significant 4 bytes and a 32-bit sub-position using +the least significant 4 bytes. For each key in the set of positions, a 32-bit +Roaring bitmap is maintained to store a set of 32-bit sub-positions for that +key. + +To test whether a certain position is set, its most significant 4 bytes (the +key) are used to find a 32-bit bitmap and the least significant 4 bytes (the +sub-position) are tested for inclusion in the bitmap. If a bitmap is not found +for the key, then it is not set. + +The serialized blob contains: +* Combined length of the vector and magic bytes stored as 4 bytes, big-endian +* A 4-byte magic sequence, `D1 D3 39 64` +* The vector, serialized as described below +* A CRC-32 checksum of the magic bytes and serialized vector as 4 bytes, big-endian + +The position vector is serialized using the Roaring bitmap +["portable" format][roaring-bitmap-portable-serialization]. This representation +consists of: + +* The number of 32-bit Roaring bitmaps, serialized as 8 bytes, little-endian +* For each 32-bit Roaring bitmap, ordered by unsigned comparison of the 32-bit keys: + - The key stored as 4 bytes, little-endian + - A [32-bit Roaring bitmap][roaring-bitmap-general-layout] + +Note that the length and CRC fields are stored using big-endian, but the +Roaring bitmap format uses little-endian values. Big endian values were chosen +for compatibility with existing deletion vectors in Delta tables. + +The blob's `properties` must: + +* Include `referenced-data-file`, the location of the data file the delete + vector applies to; must be equal to the data file's `location` in table + metadata +* Include `cardinality`, the number of deleted rows (set positions) in the + delete vector +* Omit `compression-codec`; `deletion-vector-v1` is not compressed + +Snapshot ID and sequence number are not known at the time the Puffin file is +created. `snapshot-id` and `sequence-number` must be set to -1 in blob metadata +for Puffin v1. + + +[roaring-bitmap-portable-serialization]: https://github.com/RoaringBitmap/RoaringFormatSpec?tab=readme-ov-file#extension-for-64-bit-implementations +[roaring-bitmap-general-layout]: https://github.com/RoaringBitmap/RoaringFormatSpec?tab=readme-ov-file#general-layout + ### Compression codecs The data can also be uncompressed. If it is compressed the codec should be one of From d368a5f84448b2e5698f9cab668a80eda6e3f96d Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sat, 2 Nov 2024 03:18:04 -0700 Subject: [PATCH 084/313] Spec: Add deletion vectors to the table spec (#11240) Co-authored-by: Anton Okolnychyi Co-authored-by: emkornfield --- format/spec.md | 78 +++++++++++++++++++++++++---- open-api/rest-catalog-open-api.py | 12 ++++- open-api/rest-catalog-open-api.yaml | 9 ++++ 3 files changed, 87 insertions(+), 12 deletions(-) diff --git a/format/spec.md b/format/spec.md index 6b80e876ed43..bdc328451cf2 100644 --- a/format/spec.md +++ b/format/spec.md @@ -52,6 +52,8 @@ Version 3 of the Iceberg spec extends data types and existing metadata structure * Default value support for columns * Multi-argument transforms for partitioning and sorting * Row Lineage tracking +* Binary deletion vectors + ## Goals @@ -156,6 +158,8 @@ Readers should be more permissive because v1 metadata files are allowed in v2 ta | _required_ | _optional_ | Read the field as _optional_ | | _required_ | _required_ | Fill in a default or throw an exception if the field is missing | +If a later version is not shown, the requirement for a version is not changed from the most recent version shown. For example, v3 uses the same requirements as v2 if a table shows only v1 and v2 requirements. + Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required fields that were not present in or were optional in prior versions may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. #### Writing data files @@ -567,9 +571,9 @@ The schema of a manifest file is a struct called `manifest_entry` with the follo | ---------- |------------|------------|-----------------------------------|-----------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | | _required_ | _required_ | **`134 content`** | `int` with meaning: `0: DATA`, `1: POSITION DELETES`, `2: EQUALITY DELETES` | Type of content stored by the data file: data, equality deletes, or position deletes (all v1 files are data files) | | _required_ | _required_ | _required_ | **`100 file_path`** | `string` | Full URI for the file with FS scheme | -| _required_ | _required_ | _required_ | **`101 file_format`** | `string` | String file format name, avro, orc or parquet | +| _required_ | _required_ | _required_ | **`101 file_format`** | `string` | String file format name, `avro`, `orc`, `parquet`, or `puffin` | | _required_ | _required_ | _required_ | **`102 partition`** | `struct<...>` | Partition data tuple, schema based on the partition spec output using partition field ids for the struct field ids | -| _required_ | _required_ | _required_ | **`103 record_count`** | `long` | Number of records in this file | +| _required_ | _required_ | _required_ | **`103 record_count`** | `long` | Number of records in this file, or the cardinality of a deletion vector | | _required_ | _required_ | _required_ | **`104 file_size_in_bytes`** | `long` | Total file size in bytes | | _required_ | | | ~~**`105 block_size_in_bytes`**~~ | `long` | **Deprecated. Always write a default in v1. Do not write in v2 or v3.** | | _optional_ | | | ~~**`106 file_ordinal`**~~ | `int` | **Deprecated. Do not write.** | @@ -585,13 +589,19 @@ The schema of a manifest file is a struct called `manifest_entry` with the follo | _optional_ | _optional_ | _optional_ | **`132 split_offsets`** | `list<133: long>` | Split offsets for the data file. For example, all row group offsets in a Parquet file. Must be sorted ascending | | | _optional_ | _optional_ | **`135 equality_ids`** | `list<136: int>` | Field ids used to determine row equality in equality delete files. Required when `content=2` and should be null otherwise. Fields with ids listed in this column must be present in the delete file | | _optional_ | _optional_ | _optional_ | **`140 sort_order_id`** | `int` | ID representing sort order for this file [3]. | -| | | _optional_ | **`142 first_row_id`** | `long` | The `_row_id` for the first row in the data file. See [First Row ID Inheritance](#first-row-id-inheritance) | +| | | _optional_ | **`142 first_row_id`** | `long` | The `_row_id` for the first row in the data file. See [First Row ID Inheritance](#first-row-id-inheritance) | +| | _optional_ | _optional_ | **`143 referenced_data_file`** | `string` | Fully qualified location (URI with FS scheme) of a data file that all deletes reference [4] | +| | | _optional_ | **`144 content_offset`** | `long` | The offset in the file where the content starts [5] | +| | | _optional_ | **`145 content_size_in_bytes`** | `long` | The length of a referenced content stored in the file; required if `content_offset` is present [5] | + Notes: 1. Single-value serialization for lower and upper bounds is detailed in Appendix D. 2. For `float` and `double`, the value `-0.0` must precede `+0.0`, as in the IEEE 754 `totalOrder` predicate. NaNs are not permitted as lower or upper bounds. 3. If sort order ID is missing or unknown, then the order is assumed to be unsorted. Only data files and equality delete files should be written with a non-null order id. [Position deletes](#position-delete-files) are required to be sorted by file and position, not a table order, and should set sort order id to null. Readers must ignore sort order id for position delete files. -4. The following field ids are reserved on `data_file`: 141. +4. Position delete metadata can use `referenced_data_file` when all deletes tracked by the entry are in a single data file. Setting the referenced file is required for deletion vectors. +5. The `content_offset` and `content_size_in_bytes` fields are used to reference a specific blob for direct access to a deletion vector. For deletion vectors, these values are required and must exactly match the `offset` and `length` stored in the Puffin footer for the deletion vector blob. +6. The following field ids are reserved on `data_file`: 141. The `partition` struct stores the tuple of partition values for each file. Its type is derived from the partition fields of the partition spec used to write the manifest file. In v2, the partition struct's field ids must match the ids from the partition spec. @@ -741,7 +751,7 @@ Scans are planned by reading the manifest files for the current snapshot. Delete Manifests that contain no matching files, determined using either file counts or partition summaries, may be skipped. -For each manifest, scan predicates, which filter data rows, are converted to partition predicates, which filter data and delete files. These partition predicates are used to select the data and delete files in the manifest. This conversion uses the partition spec used to write the manifest file. +For each manifest, scan predicates, which filter data rows, are converted to partition predicates, which filter partition tuples. These partition predicates are used to select relevant data files, delete files, and deletion vector metadata. Conversion uses the partition spec that was used to write the manifest file regardless of the current partition spec. Scan predicates are converted to partition predicates using an _inclusive projection_: if a scan predicate matches a row, then the partition predicate must match that row’s partition. This is called _inclusive_ [1] because rows that do not match the scan predicate may be included in the scan by the partition predicate. @@ -756,11 +766,17 @@ Data files that match the query filter must be read by the scan. Note that for any snapshot, all file paths marked with "ADDED" or "EXISTING" may appear at most once across all manifest files in the snapshot. If a file path appears more than once, the results of the scan are undefined. Reader implementations may raise an error in this case, but are not required to do so. -Delete files that match the query filter must be applied to data files at read time, limited by the scope of the delete file using the following rules. +Delete files and deletion vector metadata that match the filters must be applied to data files at read time, limited by the following scope rules. +* A deletion vector must be applied to a data file when all of the following are true: + - The data file's `file_path` is equal to the deletion vector's `referenced_data_file` + - The data file's data sequence number is _less than or equal to_ the deletion vector's data sequence number + - The data file's partition (both spec and partition values) is equal [4] to the deletion vector's partition * A _position_ delete file must be applied to a data file when all of the following are true: + - The data file's `file_path` is equal to the delete file's `referenced_data_file` if it is non-null - The data file's data sequence number is _less than or equal to_ the delete file's data sequence number - The data file's partition (both spec and partition values) is equal [4] to the delete file's partition + - There is no deletion vector that must be applied to the data file (when added, such a vector must contain all deletes from existing position delete files) * An _equality_ delete file must be applied to a data file when all of the following are true: - The data file's data sequence number is _strictly less than_ the delete's data sequence number - The data file's partition (both spec id and partition values) is equal [4] to the delete file's partition _or_ the delete file's partition spec is unpartitioned @@ -768,7 +784,7 @@ Delete files that match the query filter must be applied to data files at read t In general, deletes are applied only to data files that are older and in the same partition, except for two special cases: * Equality delete files stored with an unpartitioned spec are applied as global deletes. Otherwise, delete files do not apply to files in other partitions. -* Position delete files must be applied to data files from the same commit, when the data and delete file data sequence numbers are equal. This allows deleting rows that were added in the same commit. +* Position deletes (vectors and files) must be applied to data files from the same commit, when the data and delete file data sequence numbers are equal. This allows deleting rows that were added in the same commit. Notes: @@ -982,19 +998,45 @@ Notes: ### Delete Formats -This section details how to encode row-level deletes in Iceberg delete files. Row-level deletes are not supported in v1. +This section details how to encode row-level deletes in Iceberg delete files. Row-level deletes are added by v2 and are not supported in v1. Deletion vectors are added in v3 and are not supported in v2 or earlier. Position delete files must not be added to v3 tables, but existing position delete files are valid. + +There are three types of row-level deletes: +* Deletion vectors (DVs) identify deleted rows within a single referenced data file by position in a bitmap +* Position delete files identify deleted rows by file location and row position (**deprecated**) +* Equality delete files identify deleted rows by the value of one or more columns + +Deletion vectors are a binary representation of deletes for a single data file that is more efficient at execution time than position delete files. Unlike equality or position delete files, there can be at most one deletion vector for a given data file in a snapshot. Writers must ensure that there is at most one deletion vector per data file and must merge new deletes with existing vectors or position delete files. + +Row-level delete files (both equality and position delete files) are valid Iceberg data files: files must use valid Iceberg formats, schemas, and column projection. It is recommended that these delete files are written using the table's default file format. + +Row-level delete files and deletion vectors are tracked by manifests. A separate set of manifests is used for delete files and DVs, but the same manifest schema is used for both data and delete manifests. Deletion vectors are tracked individually by file location, offset, and length within the containing file. Deletion vector metadata must include the referenced data file. + +Both position and equality delete files allow encoding deleted row values with a delete. This can be used to reconstruct a stream of changes to a table. -Row-level delete files are valid Iceberg data files: files must use valid Iceberg formats, schemas, and column projection. It is recommended that delete files are written using the table's default file format. -Row-level delete files are tracked by manifests, like data files. A separate set of manifests is used for delete files, but the manifest schemas are identical. +### Deletion Vectors -Both position and equality deletes allow encoding deleted row values with a delete. This can be used to reconstruct a stream of changes to a table. +Deletion vectors identify deleted rows of a file by encoding deleted positions in a bitmap. A set bit at position P indicates that the row at position P is deleted. +These vectors are stored using the `deletion-vector-v1` blob definition from the [Puffin spec][puffin-spec]. + +Deletion vectors support positive 64-bit positions, but are optimized for cases where most positions fit in 32 bits by using a collection of 32-bit Roaring bitmaps. 64-bit positions are divided into a 32-bit "key" using the most significant 4 bytes and a 32-bit sub-position using the least significant 4 bytes. For each key in the set of positions, a 32-bit Roaring bitmap is maintained to store a set of 32-bit sub-positions for that key. + +To test whether a certain position is set, its most significant 4 bytes (the key) are used to find a 32-bit bitmap and the least significant 4 bytes (the sub-position) are tested for inclusion in the bitmap. If a bitmap is not found for the key, then it is not set. + +Delete manifests track deletion vectors individually by the containing file location (`file_path`), starting offset of the DV blob (`content_offset`), and total length of the blob (`content_size_in_bytes`). Multiple deletion vectors can be stored in the same file. There are no restrictions on the data files that can be referenced by deletion vectors in the same Puffin file. + +At most one deletion vector is allowed per data file in a snapshot. If a DV is written for a data file, it must replace all previously written position delete files so that when a DV is present, readers can safely ignore matching position delete files. + + +[puffin-spec]: https://iceberg.apache.org/puffin-spec/ #### Position Delete Files Position-based delete files identify deleted rows by file and position in one or more data files, and may optionally contain the deleted row. +_Note: Position delete files are **deprecated** in v3. Existing position deletes must be written to delete vectors when updating the position deletes for a data file._ + A data row is deleted if there is an entry in a position delete file for the row's file and position in the data file, starting at 0. Position-based delete files store `file_position_delete`, a struct with the following fields: @@ -1494,6 +1536,20 @@ Writing v1 or v2 metadata: * For a single-arg transform, `source-id` should be written; if `source-ids` is also written it should be a single-element list of `source-id` * For multi-arg transforms, `source-ids` should be written; `source-id` should be set to the first element of `source-ids` +Row-level delete changes: + +* Deletion vectors are added in v3, stored using the Puffin `deletion-vector-v1` blob type +* Manifests are updated to track deletion vectors: + * `referenced_data_file` was added and can be used for both deletion vectors (required) and v2 position delete files that contain deletes for only one data file (optional) + * `content_offset` was added and must match the deletion vector blob's offset in a Puffin file + * `content_size_in_bytes` was added and must match the deletion vector blob's length in a Puffin file +* Deletion vectors are maintained synchronously: Writers must merge DVs (and older position delete files) to ensure there is at most one DV per data file + * Readers can safely ignore position delete files if there is a DV for a data file +* Writers are not allowed to add new position delete files to v3 tables +* Existing position delete files are valid in tables that have been upgraded from v2 + * These position delete files must be merged into the DV for a data file when one is created + * Position delete files that contain deletes for more than one data file need to be kept in table metadata until all deletes are replaced by DVs + ### Version 2 Writing v1 metadata: diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 684e4bdb0fa6..c3372544ef95 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -830,7 +830,7 @@ class PrimitiveTypeValue(BaseModel): class FileFormat(BaseModel): - __root__: Literal['avro', 'orc', 'parquet'] + __root__: Literal['avro', 'orc', 'parquet', 'puffin'] class ContentFile(BaseModel): @@ -860,6 +860,16 @@ class ContentFile(BaseModel): class PositionDeleteFile(ContentFile): content: Literal['position-deletes'] + content_offset: Optional[int] = Field( + None, + alias='content-offset', + description='Offset within the delete file of delete content', + ) + content_size_in_bytes: Optional[int] = Field( + None, + alias='content-size-in-bytes', + description='Length, in bytes, of the delete content; required if content-offset is present', + ) class EqualityDeleteFile(ContentFile): diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index d91e32ec493e..9635af96c1ca 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -4132,6 +4132,7 @@ components: - avro - orc - parquet + - puffin ContentFile: discriminator: @@ -4241,6 +4242,14 @@ components: content: type: string enum: [ "position-deletes" ] + content-offset: + type: integer + format: int64 + description: Offset within the delete file of delete content + content-size-in-bytes: + type: integer + format: int64 + description: Length, in bytes, of the delete content; required if content-offset is present EqualityDeleteFile: allOf: From d9b9768766b359adf696f5dc9e321507bd0213d2 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Sat, 2 Nov 2024 17:23:28 +0100 Subject: [PATCH 085/313] API, Core: Add data file reference to DeleteFile (#11443) --- .../java/org/apache/iceberg/DataFile.java | 11 +++++-- .../java/org/apache/iceberg/DeleteFile.java | 11 +++++++ .../java/org/apache/iceberg/BaseFile.java | 17 +++++++++- .../java/org/apache/iceberg/BaseScan.java | 1 + .../org/apache/iceberg/ContentFileParser.java | 13 +++++++- .../java/org/apache/iceberg/FileMetadata.java | 13 +++++++- .../org/apache/iceberg/GenericDataFile.java | 3 +- .../org/apache/iceberg/GenericDeleteFile.java | 6 ++-- .../org/apache/iceberg/SnapshotProducer.java | 5 +++ .../java/org/apache/iceberg/V2Metadata.java | 9 +++++- .../java/org/apache/iceberg/V3Metadata.java | 9 +++++- .../apache/iceberg/util/ContentFileUtil.java | 4 +++ .../java/org/apache/iceberg/TestBase.java | 12 +++++++ .../apache/iceberg/TestContentFileParser.java | 32 +++++++++++++++++-- .../iceberg/TestManifestEncryption.java | 3 +- .../apache/iceberg/TestManifestReader.java | 22 +++++++++++-- .../iceberg/TestManifestWriterVersions.java | 1 + 17 files changed, 157 insertions(+), 15 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/DataFile.java b/api/src/main/java/org/apache/iceberg/DataFile.java index 02ad0aff3128..3c6d77f34d8f 100644 --- a/api/src/main/java/org/apache/iceberg/DataFile.java +++ b/api/src/main/java/org/apache/iceberg/DataFile.java @@ -98,12 +98,18 @@ public interface DataFile extends ContentFile { Types.NestedField SORT_ORDER_ID = optional(140, "sort_order_id", IntegerType.get(), "Sort order ID"); Types.NestedField SPEC_ID = optional(141, "spec_id", IntegerType.get(), "Partition spec ID"); + Types.NestedField REFERENCED_DATA_FILE = + optional( + 143, + "referenced_data_file", + StringType.get(), + "Fully qualified location (URI with FS scheme) of a data file that all deletes reference"); int PARTITION_ID = 102; String PARTITION_NAME = "partition"; String PARTITION_DOC = "Partition data tuple, schema based on the partition spec"; - // NEXT ID TO ASSIGN: 142 + // NEXT ID TO ASSIGN: 144 static StructType getType(StructType partitionType) { // IDs start at 100 to leave room for changes to ManifestEntry @@ -124,7 +130,8 @@ static StructType getType(StructType partitionType) { KEY_METADATA, SPLIT_OFFSETS, EQUALITY_IDS, - SORT_ORDER_ID); + SORT_ORDER_ID, + REFERENCED_DATA_FILE); } /** diff --git a/api/src/main/java/org/apache/iceberg/DeleteFile.java b/api/src/main/java/org/apache/iceberg/DeleteFile.java index 0f8087e6a055..8e17e60fcccf 100644 --- a/api/src/main/java/org/apache/iceberg/DeleteFile.java +++ b/api/src/main/java/org/apache/iceberg/DeleteFile.java @@ -31,4 +31,15 @@ public interface DeleteFile extends ContentFile { default List splitOffsets() { return null; } + + /** + * Returns the location of a data file that all deletes reference. + * + *

The referenced data file is required for deletion vectors and may be optionally captured for + * position delete files that apply to only one data file. This method always returns null for + * equality delete files. + */ + default String referencedDataFile() { + return null; + } } diff --git a/core/src/main/java/org/apache/iceberg/BaseFile.java b/core/src/main/java/org/apache/iceberg/BaseFile.java index 8f84eb5737b9..f4fd94724e95 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFile.java +++ b/core/src/main/java/org/apache/iceberg/BaseFile.java @@ -80,6 +80,7 @@ public PartitionData copy() { private int[] equalityIds = null; private byte[] keyMetadata = null; private Integer sortOrderId; + private String referencedDataFile = null; // cached schema private transient Schema avroSchema = null; @@ -108,6 +109,7 @@ public PartitionData copy() { DataFile.SPLIT_OFFSETS, DataFile.EQUALITY_IDS, DataFile.SORT_ORDER_ID, + DataFile.REFERENCED_DATA_FILE, MetadataColumns.ROW_POSITION); /** Used by Avro reflection to instantiate this class when reading manifest files. */ @@ -149,7 +151,8 @@ public PartitionData copy() { List splitOffsets, int[] equalityFieldIds, Integer sortOrderId, - ByteBuffer keyMetadata) { + ByteBuffer keyMetadata, + String referencedDataFile) { super(BASE_TYPE.fields().size()); this.partitionSpecId = specId; this.content = content; @@ -178,6 +181,7 @@ public PartitionData copy() { this.equalityIds = equalityFieldIds; this.sortOrderId = sortOrderId; this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); + this.referencedDataFile = referencedDataFile; } /** @@ -230,6 +234,7 @@ public PartitionData copy() { this.sortOrderId = toCopy.sortOrderId; this.dataSequenceNumber = toCopy.dataSequenceNumber; this.fileSequenceNumber = toCopy.fileSequenceNumber; + this.referencedDataFile = toCopy.referencedDataFile; } /** Constructor for Java serialization. */ @@ -339,6 +344,9 @@ protected void internalSet(int pos, T value) { this.sortOrderId = (Integer) value; return; case 17: + this.referencedDataFile = value != null ? value.toString() : null; + return; + case 18: this.fileOrdinal = (long) value; return; default: @@ -388,6 +396,8 @@ private Object getByPos(int basePos) { case 16: return sortOrderId; case 17: + return referencedDataFile; + case 18: return fileOrdinal; default: throw new UnsupportedOperationException("Unknown field ordinal: " + basePos); @@ -514,6 +524,10 @@ public Integer sortOrderId() { return sortOrderId; } + public String referencedDataFile() { + return referencedDataFile; + } + private static Map copyMap(Map map, Set keys) { return keys == null ? SerializableMap.copyOf(map) : SerializableMap.filteredCopyOf(map, keys); } @@ -565,6 +579,7 @@ public String toString() { .add("sort_order_id", sortOrderId) .add("data_sequence_number", dataSequenceNumber == null ? "null" : dataSequenceNumber) .add("file_sequence_number", fileSequenceNumber == null ? "null" : fileSequenceNumber) + .add("referenced_data_file", referencedDataFile == null ? "null" : referencedDataFile) .toString(); } } diff --git a/core/src/main/java/org/apache/iceberg/BaseScan.java b/core/src/main/java/org/apache/iceberg/BaseScan.java index 804df01d31ba..a011d03d59ad 100644 --- a/core/src/main/java/org/apache/iceberg/BaseScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseScan.java @@ -77,6 +77,7 @@ abstract class BaseScan> "partition", "key_metadata", "split_offsets", + "referenced_data_file", "equality_ids"); protected static final List DELETE_SCAN_WITH_STATS_COLUMNS = diff --git a/core/src/main/java/org/apache/iceberg/ContentFileParser.java b/core/src/main/java/org/apache/iceberg/ContentFileParser.java index dd08c5c69e7d..96dfa5586c31 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileParser.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileParser.java @@ -45,6 +45,7 @@ class ContentFileParser { private static final String SPLIT_OFFSETS = "split-offsets"; private static final String EQUALITY_IDS = "equality-ids"; private static final String SORT_ORDER_ID = "sort-order-id"; + private static final String REFERENCED_DATA_FILE = "referenced-data-file"; private ContentFileParser() {} @@ -109,6 +110,14 @@ static void toJson(ContentFile contentFile, PartitionSpec spec, JsonGenerator generator.writeNumberField(SORT_ORDER_ID, contentFile.sortOrderId()); } + if (contentFile instanceof DeleteFile) { + DeleteFile deleteFile = (DeleteFile) contentFile; + + if (deleteFile.referencedDataFile() != null) { + generator.writeStringField(REFERENCED_DATA_FILE, deleteFile.referencedDataFile()); + } + } + generator.writeEndObject(); } @@ -145,6 +154,7 @@ static ContentFile fromJson(JsonNode jsonNode, PartitionSpec spec) { List splitOffsets = JsonUtil.getLongListOrNull(SPLIT_OFFSETS, jsonNode); int[] equalityFieldIds = JsonUtil.getIntArrayOrNull(EQUALITY_IDS, jsonNode); Integer sortOrderId = JsonUtil.getIntOrNull(SORT_ORDER_ID, jsonNode); + String referencedDataFile = JsonUtil.getStringOrNull(REFERENCED_DATA_FILE, jsonNode); if (fileContent == FileContent.DATA) { return new GenericDataFile( @@ -169,7 +179,8 @@ static ContentFile fromJson(JsonNode jsonNode, PartitionSpec spec) { equalityFieldIds, sortOrderId, splitOffsets, - keyMetadata); + keyMetadata, + referencedDataFile); } } diff --git a/core/src/main/java/org/apache/iceberg/FileMetadata.java b/core/src/main/java/org/apache/iceberg/FileMetadata.java index 9a201d1b3b6f..ef229593bcab 100644 --- a/core/src/main/java/org/apache/iceberg/FileMetadata.java +++ b/core/src/main/java/org/apache/iceberg/FileMetadata.java @@ -59,6 +59,7 @@ public static class Builder { private ByteBuffer keyMetadata = null; private Integer sortOrderId = null; private List splitOffsets = null; + private String referencedDataFile = null; Builder(PartitionSpec spec) { this.spec = spec; @@ -220,6 +221,15 @@ public Builder withSortOrder(SortOrder newSortOrder) { return this; } + public Builder withReferencedDataFile(CharSequence newReferencedDataFile) { + if (newReferencedDataFile != null) { + this.referencedDataFile = newReferencedDataFile.toString(); + } else { + this.referencedDataFile = null; + } + return this; + } + public DeleteFile build() { Preconditions.checkArgument(filePath != null, "File path is required"); if (format == null) { @@ -262,7 +272,8 @@ public DeleteFile build() { equalityFieldIds, sortOrderId, splitOffsets, - keyMetadata); + keyMetadata, + referencedDataFile); } } } diff --git a/core/src/main/java/org/apache/iceberg/GenericDataFile.java b/core/src/main/java/org/apache/iceberg/GenericDataFile.java index 7b99e7b60ab8..aa34cd22cdaa 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDataFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java @@ -64,7 +64,8 @@ class GenericDataFile extends BaseFile implements DataFile { splitOffsets, null /* no equality field IDs */, sortOrderId, - keyMetadata); + keyMetadata, + null /* no referenced data file */); } /** diff --git a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java index 77e0d8505af6..05eb7c97dbab 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java @@ -48,7 +48,8 @@ class GenericDeleteFile extends BaseFile implements DeleteFile { int[] equalityFieldIds, Integer sortOrderId, List splitOffsets, - ByteBuffer keyMetadata) { + ByteBuffer keyMetadata, + String referencedDataFile) { super( specId, content, @@ -66,7 +67,8 @@ class GenericDeleteFile extends BaseFile implements DeleteFile { splitOffsets, equalityFieldIds, sortOrderId, - keyMetadata); + keyMetadata, + referencedDataFile); } /** diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 89f9eab7192a..daf1c3d72b89 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -923,5 +923,10 @@ public List equalityFieldIds() { public Integer sortOrderId() { return deleteFile.sortOrderId(); } + + @Override + public String referencedDataFile() { + return deleteFile.referencedDataFile(); + } } } diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index be4c3734e40b..20b2169b8dad 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -274,7 +274,8 @@ static Types.StructType fileType(Types.StructType partitionType) { DataFile.KEY_METADATA, DataFile.SPLIT_OFFSETS, DataFile.EQUALITY_IDS, - DataFile.SORT_ORDER_ID); + DataFile.SORT_ORDER_ID, + DataFile.REFERENCED_DATA_FILE); } static class IndexedManifestEntry> @@ -448,6 +449,12 @@ public Object get(int pos) { return wrapped.equalityFieldIds(); case 15: return wrapped.sortOrderId(); + case 16: + if (wrapped instanceof DeleteFile) { + return ((DeleteFile) wrapped).referencedDataFile(); + } else { + return null; + } } throw new IllegalArgumentException("Unknown field ordinal: " + pos); } diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java index f295af3e109d..a418a868564e 100644 --- a/core/src/main/java/org/apache/iceberg/V3Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -274,7 +274,8 @@ static Types.StructType fileType(Types.StructType partitionType) { DataFile.KEY_METADATA, DataFile.SPLIT_OFFSETS, DataFile.EQUALITY_IDS, - DataFile.SORT_ORDER_ID); + DataFile.SORT_ORDER_ID, + DataFile.REFERENCED_DATA_FILE); } static class IndexedManifestEntry> @@ -448,6 +449,12 @@ public Object get(int pos) { return wrapped.equalityFieldIds(); case 15: return wrapped.sortOrderId(); + case 16: + if (wrapped.content() == FileContent.POSITION_DELETES) { + return ((DeleteFile) wrapped).referencedDataFile(); + } else { + return null; + } } throw new IllegalArgumentException("Unknown field ordinal: " + pos); } diff --git a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java index 04fc077d10ea..c82b3ff828cf 100644 --- a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java @@ -54,6 +54,10 @@ public static CharSequence referencedDataFile(DeleteFile deleteFile) { return null; } + if (deleteFile.referencedDataFile() != null) { + return deleteFile.referencedDataFile(); + } + int pathId = MetadataColumns.DELETE_FILE_PATH.fieldId(); Type pathType = MetadataColumns.DELETE_FILE_PATH.type(); diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index f3bbb7979547..45441631900c 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -654,6 +654,18 @@ protected DeleteFile newDeleteFile(int specId, String partitionPath) { .build(); } + protected DeleteFile newDeleteFileWithRef(DataFile dataFile) { + PartitionSpec spec = table.specs().get(dataFile.specId()); + return FileMetadata.deleteFileBuilder(spec) + .ofPositionDeletes() + .withPath("/path/to/delete-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(10) + .withPartition(dataFile.partition()) + .withReferencedDataFile(dataFile.location()) + .withRecordCount(1) + .build(); + } + protected DeleteFile newEqualityDeleteFile(int specId, String partitionPath, int... fieldIds) { PartitionSpec spec = table.specs().get(specId); return FileMetadata.deleteFileBuilder(spec) diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index 83f7fc1f6220..fbe473931659 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -213,7 +213,33 @@ private static Stream provideSpecAndDeleteFile() { Arguments.of( TestBase.SPEC, deleteFileWithAllOptional(TestBase.SPEC), - deleteFileJsonWithAllOptional(TestBase.SPEC))); + deleteFileJsonWithAllOptional(TestBase.SPEC)), + Arguments.of( + TestBase.SPEC, deleteFileWithDataRef(TestBase.SPEC), deleteFileWithDataRefJson())); + } + + private static DeleteFile deleteFileWithDataRef(PartitionSpec spec) { + PartitionData partitionData = new PartitionData(spec.partitionType()); + partitionData.set(0, 4); + return new GenericDeleteFile( + spec.specId(), + FileContent.POSITION_DELETES, + "/path/to/delete.parquet", + FileFormat.PARQUET, + partitionData, + 1234, + new Metrics(10L, null, null, null, null), + null, + null, + null, + null, + "/path/to/data/file.parquet"); + } + + private static String deleteFileWithDataRefJson() { + return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":4},\"file-size-in-bytes\":1234," + + "\"record-count\":10,\"referenced-data-file\":\"/path/to/data/file.parquet\"}"; } private static DeleteFile deleteFileWithRequiredOnly(PartitionSpec spec) { @@ -234,6 +260,7 @@ private static DeleteFile deleteFileWithRequiredOnly(PartitionSpec spec) { null, null, null, + null, null); } @@ -273,7 +300,8 @@ private static DeleteFile deleteFileWithAllOptional(PartitionSpec spec) { new int[] {3}, 1, Collections.singletonList(128L), - ByteBuffer.wrap(new byte[16])); + ByteBuffer.wrap(new byte[16]), + null); } private static String deleteFileJsonWithRequiredOnly(PartitionSpec spec) { diff --git a/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java index 13e8985cdb56..1f29c0e5b85c 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java @@ -110,7 +110,8 @@ public class TestManifestEncryption { EQUALITY_ID_ARR, SORT_ORDER_ID, null, - CONTENT_KEY_METADATA); + CONTENT_KEY_METADATA, + null); private static final EncryptionManager ENCRYPTION_MANAGER = EncryptionTestHelpers.createEncryptionManager(); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java index e45415f1f2d2..4652da943003 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java @@ -130,7 +130,7 @@ public void testDataFilePositions() throws IOException { long expectedPos = 0L; for (DataFile file : reader) { assertThat(file.pos()).as("Position should match").isEqualTo(expectedPos); - assertThat(((BaseFile) file).get(17)) + assertThat(((BaseFile) file).get(18)) .as("Position from field index should match") .isEqualTo(expectedPos); expectedPos += 1; @@ -158,7 +158,7 @@ public void testDeleteFilePositions() throws IOException { long expectedPos = 0L; for (DeleteFile file : reader) { assertThat(file.pos()).as("Position should match").isEqualTo(expectedPos); - assertThat(((BaseFile) file).get(17)) + assertThat(((BaseFile) file).get(18)) .as("Position from field index should match") .isEqualTo(expectedPos); expectedPos += 1; @@ -181,6 +181,24 @@ public void testDeleteFileManifestPaths() throws IOException { } } + @TestTemplate + public void testDeleteFilesWithReferences() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + DeleteFile deleteFile1 = newDeleteFileWithRef(FILE_A); + DeleteFile deleteFile2 = newDeleteFileWithRef(FILE_B); + ManifestFile manifest = writeDeleteManifest(formatVersion, 1000L, deleteFile1, deleteFile2); + try (ManifestReader reader = + ManifestFiles.readDeleteManifest(manifest, FILE_IO, table.specs())) { + for (DeleteFile deleteFile : reader) { + if (deleteFile.location().equals(deleteFile1.location())) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(FILE_A.location()); + } else { + assertThat(deleteFile.referencedDataFile()).isEqualTo(FILE_B.location()); + } + } + } + } + @TestTemplate public void testDataFileSplitOffsetsNullWhenInvalid() throws IOException { DataFile invalidOffset = diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 1d5c34fa4b16..88dcc6ff9ca4 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -96,6 +96,7 @@ public class TestManifestWriterVersions { EQUALITY_ID_ARR, SORT_ORDER_ID, null, + null, null); @TempDir private Path temp; From e47fa6a581f1396c35b13ead228e886003c9dfdf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 08:49:46 +0100 Subject: [PATCH 086/313] Build: Bump software.amazon.awssdk:bom from 2.29.1 to 2.29.6 (#11454) Bumps software.amazon.awssdk:bom from 2.29.1 to 2.29.6. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 066c3549769f..cbc0a3b94752 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.29.1" +awssdk-bom = "2.29.6" azuresdk-bom = "1.2.28" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" From 29ee906496d840785f814f1aee99eb0e0767f0ae Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 08:50:16 +0100 Subject: [PATCH 087/313] Build: Bump com.gradleup.shadow:shadow-gradle-plugin from 8.3.3 to 8.3.5 (#11452) Bumps [com.gradleup.shadow:shadow-gradle-plugin](https://github.com/GradleUp/shadow) from 8.3.3 to 8.3.5. - [Release notes](https://github.com/GradleUp/shadow/releases) - [Commits](https://github.com/GradleUp/shadow/compare/8.3.3...8.3.5) --- updated-dependencies: - dependency-name: com.gradleup.shadow:shadow-gradle-plugin dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 1a05f83f7d9b..ad4d77f9d161 100644 --- a/build.gradle +++ b/build.gradle @@ -26,7 +26,7 @@ buildscript { gradlePluginPortal() } dependencies { - classpath 'com.gradleup.shadow:shadow-gradle-plugin:8.3.3' + classpath 'com.gradleup.shadow:shadow-gradle-plugin:8.3.5' classpath 'com.palantir.baseline:gradle-baseline-java:5.72.0' classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.25.0' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' From 7ffb6a3de78014c9917eaafbda947a3ebcdd1389 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 08:50:40 +0100 Subject: [PATCH 088/313] Build: Bump com.google.cloud:libraries-bom from 26.49.0 to 26.50.0 (#11451) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.49.0 to 26.50.0. - [Release notes](https://github.com/googleapis/java-cloud-bom/releases) - [Changelog](https://github.com/googleapis/java-cloud-bom/blob/main/release-please-config.json) - [Commits](https://github.com/googleapis/java-cloud-bom/compare/v26.49.0...v26.50.0) --- updated-dependencies: - dependency-name: com.google.cloud:libraries-bom dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index cbc0a3b94752..d48fb471ad00 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -44,7 +44,7 @@ findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} flink120 = { strictly = "1.20.0"} -google-libraries-bom = "26.49.0" +google-libraries-bom = "26.50.0" guava = "33.3.1-jre" hadoop2 = "2.7.3" hadoop3 = "3.4.1" From aa27d9057d7471e4ca9d1e03e20d9cbbf0c87890 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 08:50:56 +0100 Subject: [PATCH 089/313] Build: Bump org.apache.httpcomponents.client5:httpclient5 (#11450) Bumps [org.apache.httpcomponents.client5:httpclient5](https://github.com/apache/httpcomponents-client) from 5.4 to 5.4.1. - [Changelog](https://github.com/apache/httpcomponents-client/blob/rel/v5.4.1/RELEASE_NOTES.txt) - [Commits](https://github.com/apache/httpcomponents-client/compare/rel/v5.4...rel/v5.4.1) --- updated-dependencies: - dependency-name: org.apache.httpcomponents.client5:httpclient5 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index d48fb471ad00..05cea6320262 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -48,7 +48,7 @@ google-libraries-bom = "26.50.0" guava = "33.3.1-jre" hadoop2 = "2.7.3" hadoop3 = "3.4.1" -httpcomponents-httpclient5 = "5.4" +httpcomponents-httpclient5 = "5.4.1" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" immutables-value = "2.10.1" From c7f0f80e030866b4bb6795a20c50d9c1c2ed726b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 11:39:56 +0100 Subject: [PATCH 090/313] Build: Bump com.azure:azure-sdk-bom from 1.2.28 to 1.2.29 (#11453) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.2.28 to 1.2.29. - [Release notes](https://github.com/azure/azure-sdk-for-java/releases) - [Commits](https://github.com/azure/azure-sdk-for-java/compare/azure-sdk-bom_1.2.28...azure-sdk-bom_1.2.29) --- updated-dependencies: - dependency-name: com.azure:azure-sdk-bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 05cea6320262..b8c69377958e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -30,7 +30,7 @@ avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" awssdk-bom = "2.29.6" -azuresdk-bom = "1.2.28" +azuresdk-bom = "1.2.29" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" calcite = "1.10.0" From 0669bcbad761cf9ac6d4847e8f5a510f4edc8e1f Mon Sep 17 00:00:00 2001 From: pvary Date: Mon, 4 Nov 2024 15:06:00 +0100 Subject: [PATCH 091/313] Flink: Maintenance - TableManager + ExpireSnapshots (#11144) --- .../maintenance/api/ExpireSnapshots.java | 125 +++++ .../{operator => api}/JdbcLockFactory.java | 7 +- .../api/MaintenanceTaskBuilder.java | 227 +++++++++ .../maintenance/api/TableMaintenance.java | 330 +++++++++++++ .../{operator => api}/TaskResult.java | 7 +- .../{operator => api}/Trigger.java | 31 +- .../{operator => api}/TriggerLockFactory.java | 9 +- .../operator/DeleteFilesProcessor.java | 130 +++++ .../operator/ExpireSnapshotsProcessor.java | 129 +++++ .../maintenance/operator/LockRemover.java | 25 +- .../maintenance/operator/MonitorSource.java | 4 +- .../maintenance/operator/TableChange.java | 10 +- .../operator/TableMaintenanceMetrics.java | 10 +- .../operator/TriggerEvaluator.java | 8 +- .../maintenance/operator/TriggerManager.java | 54 +- .../api/MaintenanceTaskInfraExtension.java | 78 +++ .../api/MaintenanceTaskTestBase.java | 64 +++ .../maintenance/api/TestExpireSnapshots.java | 254 ++++++++++ .../TestJdbcLockFactory.java | 4 +- .../TestLockFactoryBase.java | 2 +- .../maintenance/api/TestMaintenanceE2E.java | 67 +++ .../maintenance/api/TestTableMaintenance.java | 460 ++++++++++++++++++ .../maintenance/operator/CollectingSink.java | 6 +- .../operator/ConstantsForTests.java | 29 -- .../operator/FlinkSqlExtension.java | 135 ----- .../operator/FlinkStreamingTestUtils.java | 73 --- .../maintenance/operator/ManualSource.java | 12 +- .../MetricsReporterFactoryForTests.java | 65 ++- .../operator/OperatorTestBase.java | 184 +++++-- .../operator/TestDeleteFilesProcessor.java | 116 +++++ .../TestExpireSnapshotsProcessor.java | 80 +++ .../maintenance/operator/TestLockRemover.java | 120 +++-- .../operator/TestMonitorSource.java | 72 +-- .../operator/TestTriggerManager.java | 152 +++--- 34 files changed, 2527 insertions(+), 552 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/JdbcLockFactory.java (98%) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/TaskResult.java (93%) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/Trigger.java (67%) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/TriggerLockFactory.java (86%) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/{operator => api}/TestJdbcLockFactory.java (92%) rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/{operator => api}/TestLockFactoryBase.java (97%) create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java delete mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java delete mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java delete mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java new file mode 100644 index 000000000000..9cde5cb173e1 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java @@ -0,0 +1,125 @@ +/* + * 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.flink.maintenance.api; + +import java.time.Duration; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.flink.maintenance.operator.DeleteFilesProcessor; +import org.apache.iceberg.flink.maintenance.operator.ExpireSnapshotsProcessor; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Deletes expired snapshots and the corresponding files. */ +public class ExpireSnapshots { + private static final int DELETE_BATCH_SIZE_DEFAULT = 1000; + private static final String EXECUTOR_OPERATOR_NAME = "Expire Snapshot"; + @VisibleForTesting static final String DELETE_FILES_OPERATOR_NAME = "Delete file"; + + private ExpireSnapshots() {} + + /** Creates the builder for creating a stream which expires snapshots for the table. */ + public static Builder builder() { + return new Builder(); + } + + public static class Builder extends MaintenanceTaskBuilder { + private Duration maxSnapshotAge = null; + private Integer numSnapshots = null; + private Integer planningWorkerPoolSize; + private int deleteBatchSize = DELETE_BATCH_SIZE_DEFAULT; + + /** + * The snapshots older than this age will be removed. + * + * @param newMaxSnapshotAge of the snapshots to be removed + */ + public Builder maxSnapshotAge(Duration newMaxSnapshotAge) { + this.maxSnapshotAge = newMaxSnapshotAge; + return this; + } + + /** + * The minimum number of {@link Snapshot}s to retain. For more details description see {@link + * org.apache.iceberg.ExpireSnapshots#retainLast(int)}. + * + * @param newNumSnapshots number of snapshots to retain + */ + public Builder retainLast(int newNumSnapshots) { + this.numSnapshots = newNumSnapshots; + return this; + } + + /** + * The worker pool size used to calculate the files to delete. If not set, the shared worker + * pool is used. + * + * @param newPlanningWorkerPoolSize for planning files to delete + */ + public Builder planningWorkerPoolSize(int newPlanningWorkerPoolSize) { + this.planningWorkerPoolSize = newPlanningWorkerPoolSize; + return this; + } + + /** + * Size of the batch used to deleting the files. + * + * @param newDeleteBatchSize used for deleting + */ + public Builder deleteBatchSize(int newDeleteBatchSize) { + this.deleteBatchSize = newDeleteBatchSize; + return this; + } + + @Override + DataStream append(DataStream trigger) { + Preconditions.checkNotNull(tableLoader(), "TableLoader should not be null"); + + SingleOutputStreamOperator result = + trigger + .process( + new ExpireSnapshotsProcessor( + tableLoader(), + maxSnapshotAge == null ? null : maxSnapshotAge.toMillis(), + numSnapshots, + planningWorkerPoolSize)) + .name(operatorName(EXECUTOR_OPERATOR_NAME)) + .uid(EXECUTOR_OPERATOR_NAME + uidSuffix()) + .slotSharingGroup(slotSharingGroup()) + .forceNonParallel(); + + result + .getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM) + .rebalance() + .transform( + operatorName(DELETE_FILES_OPERATOR_NAME), + TypeInformation.of(Void.class), + new DeleteFilesProcessor( + index(), taskName(), tableLoader().loadTable(), deleteBatchSize)) + .uid(DELETE_FILES_OPERATOR_NAME + uidSuffix()) + .slotSharingGroup(slotSharingGroup()) + .setParallelism(parallelism()); + + // Ignore the file deletion result and return the DataStream directly + return result; + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java similarity index 98% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java index 085fbfecd270..9f4fb069aae8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import java.io.IOException; import java.sql.DatabaseMetaData; @@ -38,10 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * JDBC table backed implementation of the {@link - * org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory}. - */ +/** JDBC table backed implementation of the {@link TriggerLockFactory}. */ public class JdbcLockFactory implements TriggerLockFactory { private static final Logger LOG = LoggerFactory.getLogger(JdbcLockFactory.class); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java new file mode 100644 index 000000000000..3fc431d02547 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -0,0 +1,227 @@ +/* + * 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.flink.maintenance.api; + +import java.time.Duration; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.TriggerEvaluator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Experimental +@SuppressWarnings("unchecked") +public abstract class MaintenanceTaskBuilder> { + private int index; + private String taskName; + private String tableName; + private TableLoader tableLoader; + private String uidSuffix = null; + private String slotSharingGroup = null; + private Integer parallelism = null; + private final TriggerEvaluator.Builder triggerEvaluator = new TriggerEvaluator.Builder(); + + abstract DataStream append(DataStream sourceStream); + + /** + * After a given number of Iceberg table commits since the last run, starts the downstream job. + * + * @param commitCount after the downstream job should be started + */ + public T scheduleOnCommitCount(int commitCount) { + triggerEvaluator.commitCount(commitCount); + return (T) this; + } + + /** + * After a given number of new data files since the last run, starts the downstream job. + * + * @param dataFileCount after the downstream job should be started + */ + public T scheduleOnDataFileCount(int dataFileCount) { + triggerEvaluator.dataFileCount(dataFileCount); + return (T) this; + } + + /** + * After a given aggregated data file size since the last run, starts the downstream job. + * + * @param dataFileSizeInBytes after the downstream job should be started + */ + public T scheduleOnDataFileSize(long dataFileSizeInBytes) { + triggerEvaluator.dataFileSizeInBytes(dataFileSizeInBytes); + return (T) this; + } + + /** + * After a given number of new positional delete files since the last run, starts the downstream + * job. + * + * @param posDeleteFileCount after the downstream job should be started + */ + public T scheduleOnPosDeleteFileCount(int posDeleteFileCount) { + triggerEvaluator.posDeleteFileCount(posDeleteFileCount); + return (T) this; + } + + /** + * After a given number of new positional delete records since the last run, starts the downstream + * job. + * + * @param posDeleteRecordCount after the downstream job should be started + */ + public T scheduleOnPosDeleteRecordCount(long posDeleteRecordCount) { + triggerEvaluator.posDeleteRecordCount(posDeleteRecordCount); + return (T) this; + } + + /** + * After a given number of new equality delete files since the last run, starts the downstream + * job. + * + * @param eqDeleteFileCount after the downstream job should be started + */ + public T scheduleOnEqDeleteFileCount(int eqDeleteFileCount) { + triggerEvaluator.eqDeleteFileCount(eqDeleteFileCount); + return (T) this; + } + + /** + * After a given number of new equality delete records since the last run, starts the downstream + * job. + * + * @param eqDeleteRecordCount after the downstream job should be started + */ + public T scheduleOnEqDeleteRecordCount(long eqDeleteRecordCount) { + triggerEvaluator.eqDeleteRecordCount(eqDeleteRecordCount); + return (T) this; + } + + /** + * After a given time since the last run, starts the downstream job. + * + * @param interval after the downstream job should be started + */ + public T scheduleOnInterval(Duration interval) { + triggerEvaluator.timeout(interval); + return (T) this; + } + + /** + * The suffix used for the generated {@link org.apache.flink.api.dag.Transformation}'s uid. + * + * @param newUidSuffix for the transformations + */ + public T uidSuffix(String newUidSuffix) { + this.uidSuffix = newUidSuffix; + return (T) this; + } + + /** + * The {@link SingleOutputStreamOperator#slotSharingGroup(String)} for all the operators of the + * generated stream. Could be used to separate the resources used by this task. + * + * @param newSlotSharingGroup to be used for the operators + */ + public T slotSharingGroup(String newSlotSharingGroup) { + this.slotSharingGroup = newSlotSharingGroup; + return (T) this; + } + + /** + * Sets the parallelism for the stream. + * + * @param newParallelism the required parallelism + */ + public T parallelism(int newParallelism) { + OperatorValidationUtils.validateParallelism(newParallelism); + this.parallelism = newParallelism; + return (T) this; + } + + protected int index() { + return index; + } + + protected String taskName() { + return taskName; + } + + protected String tableName() { + return tableName; + } + + protected TableLoader tableLoader() { + return tableLoader; + } + + protected String uidSuffix() { + return uidSuffix; + } + + protected String slotSharingGroup() { + return slotSharingGroup; + } + + protected Integer parallelism() { + return parallelism; + } + + protected String operatorName(String operatorNameBase) { + return operatorNameBase + "[" + index() + "]"; + } + + TriggerEvaluator evaluator() { + return triggerEvaluator.build(); + } + + DataStream append( + DataStream sourceStream, + int taskIndex, + String newTaskName, + String newTableName, + TableLoader newTableLoader, + String defaultUidSuffix, + String defaultSlotSharingGroup, + int defaultParallelism) { + Preconditions.checkNotNull(newTaskName, "Task name should not be null"); + Preconditions.checkNotNull(newTableLoader, "TableLoader should not be null"); + + this.index = taskIndex; + this.taskName = newTaskName; + this.tableName = newTableName; + this.tableLoader = newTableLoader; + + if (uidSuffix == null) { + uidSuffix = this.taskName + "_" + index + "_" + defaultUidSuffix; + } + + if (parallelism == null) { + parallelism = defaultParallelism; + } + + if (slotSharingGroup == null) { + slotSharingGroup = defaultSlotSharingGroup; + } + + return append(sourceStream); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java new file mode 100644 index 000000000000..f931a9127137 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java @@ -0,0 +1,330 @@ +/* + * 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.flink.maintenance.api; + +import java.io.IOException; +import java.time.Duration; +import java.util.List; +import java.util.UUID; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.eventtime.TimestampAssigner; +import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; +import org.apache.flink.api.common.eventtime.WatermarkOutput; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.LockRemover; +import org.apache.iceberg.flink.maintenance.operator.MonitorSource; +import org.apache.iceberg.flink.maintenance.operator.TableChange; +import org.apache.iceberg.flink.maintenance.operator.TriggerEvaluator; +import org.apache.iceberg.flink.maintenance.operator.TriggerManager; +import org.apache.iceberg.flink.sink.IcebergSink; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Creates the table maintenance graph. */ +public class TableMaintenance { + static final String SOURCE_OPERATOR_NAME_PREFIX = "Monitor source for "; + static final String TRIGGER_MANAGER_OPERATOR_NAME = "Trigger manager"; + static final String WATERMARK_ASSIGNER_OPERATOR_NAME = "Watermark Assigner"; + static final String FILTER_OPERATOR_NAME_PREFIX = "Filter "; + static final String LOCK_REMOVER_OPERATOR_NAME = "Lock remover"; + + private TableMaintenance() {} + + /** + * Use when the change stream is already provided, like in the {@link + * IcebergSink#addPostCommitTopology(DataStream)}. + * + * @param changeStream the table changes + * @param tableLoader used for accessing the table + * @param lockFactory used for preventing concurrent task runs + * @return builder for the maintenance stream + */ + @Internal + public static Builder forChangeStream( + DataStream changeStream, + TableLoader tableLoader, + TriggerLockFactory lockFactory) { + Preconditions.checkNotNull(changeStream, "The change stream should not be null"); + Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); + Preconditions.checkNotNull(lockFactory, "LockFactory should not be null"); + + return new Builder(null, changeStream, tableLoader, lockFactory); + } + + /** + * Use this for standalone maintenance job. It creates a monitor source that detect table changes + * and build the maintenance pipelines afterwards. + * + * @param env used to register the monitor source + * @param tableLoader used for accessing the table + * @param lockFactory used for preventing concurrent task runs + * @return builder for the maintenance stream + */ + public static Builder forTable( + StreamExecutionEnvironment env, TableLoader tableLoader, TriggerLockFactory lockFactory) { + Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); + Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); + Preconditions.checkNotNull(lockFactory, "LockFactory should not be null"); + + return new Builder(env, null, tableLoader, lockFactory); + } + + public static class Builder { + private final StreamExecutionEnvironment env; + private final DataStream inputStream; + private final TableLoader tableLoader; + private final List> taskBuilders; + private final TriggerLockFactory lockFactory; + + private String uidSuffix = "TableMaintenance-" + UUID.randomUUID(); + private String slotSharingGroup = StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP; + private Duration rateLimit = Duration.ofMinutes(1); + private Duration lockCheckDelay = Duration.ofSeconds(30); + private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT; + private int maxReadBack = 100; + + private Builder( + StreamExecutionEnvironment env, + DataStream inputStream, + TableLoader tableLoader, + TriggerLockFactory lockFactory) { + this.env = env; + this.inputStream = inputStream; + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.taskBuilders = Lists.newArrayListWithCapacity(4); + } + + /** + * The suffix used for the generated {@link Transformation}'s uid. + * + * @param newUidSuffix for the transformations + */ + public Builder uidSuffix(String newUidSuffix) { + this.uidSuffix = newUidSuffix; + return this; + } + + /** + * The {@link SingleOutputStreamOperator#slotSharingGroup(String)} for all the operators of the + * generated stream. Could be used to separate the resources used by this task. + * + * @param newSlotSharingGroup to be used for the operators + */ + public Builder slotSharingGroup(String newSlotSharingGroup) { + this.slotSharingGroup = newSlotSharingGroup; + return this; + } + + /** + * Limits the firing frequency for the task triggers. + * + * @param newRateLimit firing frequency + */ + public Builder rateLimit(Duration newRateLimit) { + Preconditions.checkNotNull(rateLimit.toMillis() > 0, "Rate limit should be greater than 0"); + this.rateLimit = newRateLimit; + return this; + } + + /** + * Sets the delay for checking lock availability when a concurrent run is detected. + * + * @param newLockCheckDelay lock checking frequency + */ + public Builder lockCheckDelay(Duration newLockCheckDelay) { + this.lockCheckDelay = newLockCheckDelay; + return this; + } + + /** + * Sets the default parallelism of maintenance tasks. Could be overwritten by the {@link + * MaintenanceTaskBuilder#parallelism(int)}. + * + * @param newParallelism task parallelism + */ + public Builder parallelism(int newParallelism) { + OperatorValidationUtils.validateParallelism(newParallelism); + this.parallelism = newParallelism; + return this; + } + + /** + * Maximum number of snapshots checked when started with an embedded {@link MonitorSource} at + * the first time. Only available when the {@link + * TableMaintenance#forTable(StreamExecutionEnvironment, TableLoader, TriggerLockFactory)} is + * used. + * + * @param newMaxReadBack snapshots to consider when initializing + */ + public Builder maxReadBack(int newMaxReadBack) { + Preconditions.checkArgument( + inputStream == null, "Can't set maxReadBack when change stream is provided"); + this.maxReadBack = newMaxReadBack; + return this; + } + + /** + * Adds a specific task with the given schedule. + * + * @param task to add + */ + public Builder add(MaintenanceTaskBuilder task) { + taskBuilders.add(task); + return this; + } + + /** Builds the task graph for the maintenance tasks. */ + public void append() throws IOException { + Preconditions.checkArgument(!taskBuilders.isEmpty(), "Provide at least one task"); + Preconditions.checkNotNull(uidSuffix, "Uid suffix should no be null"); + + List taskNames = Lists.newArrayListWithCapacity(taskBuilders.size()); + List evaluators = Lists.newArrayListWithCapacity(taskBuilders.size()); + for (int i = 0; i < taskBuilders.size(); ++i) { + taskNames.add(nameFor(taskBuilders.get(i), i)); + evaluators.add(taskBuilders.get(i).evaluator()); + } + + try (TableLoader loader = tableLoader.clone()) { + loader.open(); + String tableName = loader.loadTable().name(); + DataStream triggers = + DataStreamUtils.reinterpretAsKeyedStream( + changeStream(tableName, loader), unused -> true) + .process( + new TriggerManager( + loader, + lockFactory, + taskNames, + evaluators, + rateLimit.toMillis(), + lockCheckDelay.toMillis())) + .name(TRIGGER_MANAGER_OPERATOR_NAME) + .uid(TRIGGER_MANAGER_OPERATOR_NAME + uidSuffix) + .slotSharingGroup(slotSharingGroup) + .forceNonParallel() + .assignTimestampsAndWatermarks(new PunctuatedWatermarkStrategy()) + .name(WATERMARK_ASSIGNER_OPERATOR_NAME) + .uid(WATERMARK_ASSIGNER_OPERATOR_NAME + uidSuffix) + .slotSharingGroup(slotSharingGroup) + .forceNonParallel(); + + // Add the specific tasks + DataStream unioned = null; + for (int i = 0; i < taskBuilders.size(); ++i) { + int taskIndex = i; + DataStream filtered = + triggers + .filter(t -> t.taskId() != null && t.taskId() == taskIndex) + .name(FILTER_OPERATOR_NAME_PREFIX + taskIndex) + .forceNonParallel() + .uid(FILTER_OPERATOR_NAME_PREFIX + taskIndex + "-" + uidSuffix) + .slotSharingGroup(slotSharingGroup); + MaintenanceTaskBuilder builder = taskBuilders.get(taskIndex); + DataStream result = + builder.append( + filtered, + taskIndex, + taskNames.get(taskIndex), + tableName, + loader, + uidSuffix, + slotSharingGroup, + parallelism); + if (unioned == null) { + unioned = result; + } else { + unioned = unioned.union(result); + } + } + + // Add the LockRemover to the end + unioned + .transform( + LOCK_REMOVER_OPERATOR_NAME, + TypeInformation.of(Void.class), + new LockRemover(tableName, lockFactory, taskNames)) + .forceNonParallel() + .uid("lock-remover-" + uidSuffix) + .slotSharingGroup(slotSharingGroup); + } + } + + private DataStream changeStream(String tableName, TableLoader loader) { + if (inputStream == null) { + // Create a monitor source to provide the TableChange stream + MonitorSource source = + new MonitorSource( + loader, RateLimiterStrategy.perSecond(1.0 / rateLimit.getSeconds()), maxReadBack); + return env.fromSource( + source, WatermarkStrategy.noWatermarks(), SOURCE_OPERATOR_NAME_PREFIX + tableName) + .uid(SOURCE_OPERATOR_NAME_PREFIX + uidSuffix) + .slotSharingGroup(slotSharingGroup) + .forceNonParallel(); + } else { + return inputStream.global(); + } + } + + private static String nameFor(MaintenanceTaskBuilder streamBuilder, int taskIndex) { + return String.format( + "%s [%s]", streamBuilder.getClass().getSimpleName(), String.valueOf(taskIndex)); + } + } + + @Internal + public static class PunctuatedWatermarkStrategy implements WatermarkStrategy { + @Override + public WatermarkGenerator createWatermarkGenerator( + WatermarkGeneratorSupplier.Context context) { + return new WatermarkGenerator<>() { + @Override + public void onEvent(Trigger event, long eventTimestamp, WatermarkOutput output) { + output.emitWatermark(new Watermark(event.timestamp())); + } + + @Override + public void onPeriodicEmit(WatermarkOutput output) { + // No periodic watermarks + } + }; + } + + @Override + public TimestampAssigner createTimestampAssigner( + TimestampAssignerSupplier.Context context) { + return (element, unused) -> element.timestamp(); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java similarity index 93% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java index 06f10f1c1d68..ca1462526f13 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java @@ -16,15 +16,14 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; +import java.io.Serializable; import java.util.List; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; /** The result of a single Maintenance Task. */ -@Internal -public class TaskResult { +public class TaskResult implements Serializable { private final int taskIndex; private final long startEpoch; private final boolean success; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java similarity index 67% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java index 85c6c8dbdd55..09209ba15153 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java @@ -16,47 +16,41 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import org.apache.flink.annotation.Internal; -import org.apache.iceberg.SerializableTable; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -@Internal -class Trigger { +public class Trigger { private final long timestamp; - private final SerializableTable table; private final Integer taskId; private final boolean isRecovery; - private Trigger(long timestamp, SerializableTable table, Integer taskId, boolean isRecovery) { + private Trigger(long timestamp, Integer taskId, boolean isRecovery) { this.timestamp = timestamp; - this.table = table; this.taskId = taskId; this.isRecovery = isRecovery; } - static Trigger create(long timestamp, SerializableTable table, int taskId) { - return new Trigger(timestamp, table, taskId, false); + @Internal + public static Trigger create(long timestamp, int taskId) { + return new Trigger(timestamp, taskId, false); } - static Trigger recovery(long timestamp) { - return new Trigger(timestamp, null, null, true); + @Internal + public static Trigger recovery(long timestamp) { + return new Trigger(timestamp, null, true); } - long timestamp() { + public long timestamp() { return timestamp; } - SerializableTable table() { - return table; - } - - Integer taskId() { + public Integer taskId() { return taskId; } - boolean isRecovery() { + public boolean isRecovery() { return isRecovery; } @@ -64,7 +58,6 @@ boolean isRecovery() { public String toString() { return MoreObjects.toStringHelper(this) .add("timestamp", timestamp) - .add("table", table == null ? null : table.name()) .add("taskId", taskId) .add("isRecovery", isRecovery) .toString(); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java similarity index 86% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java index 329223d27ccf..c31381355efe 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java @@ -16,11 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import java.io.Closeable; import java.io.Serializable; import org.apache.flink.annotation.Experimental; +import org.apache.iceberg.flink.maintenance.operator.LockRemover; +import org.apache.iceberg.flink.maintenance.operator.TriggerManager; /** Lock interface for handling locks for the Flink Table Maintenance jobs. */ @Experimental @@ -51,12 +53,11 @@ interface Lock { */ boolean isHeld(); - // TODO: Fix the link to the LockRemover when we have a final name and implementation /** * Releases the lock. Should not fail if the lock is not held by anyone. * - *

Called by LockRemover. Implementations could assume that are no concurrent calls for this - * method. + *

Called by {@link LockRemover}. Implementations could assume that are no concurrent calls + * for this method. */ void unlock(); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java new file mode 100644 index 000000000000..dc7846c4c4d3 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java @@ -0,0 +1,130 @@ +/* + * 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.flink.maintenance.operator; + +import java.util.Set; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.BulkDeletionFailureException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.SupportsBulkOperations; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Delete the files using the {@link FileIO} which implements {@link SupportsBulkOperations}. */ +@Internal +public class DeleteFilesProcessor extends AbstractStreamOperator + implements OneInputStreamOperator { + private static final Logger LOG = LoggerFactory.getLogger(DeleteFilesProcessor.class); + + private final String taskIndex; + private final String taskName; + private final SupportsBulkOperations io; + private final String tableName; + private final Set filesToDelete = Sets.newHashSet(); + private final int batchSize; + + private transient Counter failedCounter; + private transient Counter succeededCounter; + + public DeleteFilesProcessor(int taskIndex, String taskName, Table table, int batchSize) { + Preconditions.checkNotNull(taskName, "Task name should no be null"); + Preconditions.checkNotNull(table, "Table should no be null"); + + FileIO fileIO = table.io(); + Preconditions.checkArgument( + fileIO instanceof SupportsBulkOperations, + "%s doesn't support bulk delete", + fileIO.getClass().getSimpleName()); + + this.taskIndex = String.valueOf(taskIndex); + this.taskName = taskName; + this.io = (SupportsBulkOperations) fileIO; + this.tableName = table.name(); + this.batchSize = batchSize; + } + + @Override + public void open() throws Exception { + this.failedCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, taskName) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, taskIndex) + .counter(TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER); + this.succeededCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, taskName) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, taskIndex) + .counter(TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + if (element.isRecord()) { + filesToDelete.add(element.getValue()); + } + + if (filesToDelete.size() >= batchSize) { + deleteFiles(); + } + } + + @Override + public void processWatermark(Watermark mark) { + deleteFiles(); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) { + deleteFiles(); + } + + private void deleteFiles() { + try { + io.deleteFiles(filesToDelete); + LOG.info( + "Deleted {} files from table {} using bulk deletes", filesToDelete.size(), tableName); + succeededCounter.inc(filesToDelete.size()); + filesToDelete.clear(); + } catch (BulkDeletionFailureException e) { + int deletedFilesCount = filesToDelete.size() - e.numberFailedObjects(); + LOG.warn( + "Deleted only {} of {} files from table {} using bulk deletes", + deletedFilesCount, + filesToDelete.size(), + tableName, + e); + succeededCounter.inc(deletedFilesCount); + failedCounter.inc(e.numberFailedObjects()); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java new file mode 100644 index 000000000000..a09d0244e95d --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -0,0 +1,129 @@ +/* + * 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.flink.maintenance.operator; + +import java.util.Collections; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; +import org.apache.iceberg.ExpireSnapshots; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Calls the {@link ExpireSnapshots} to remove the old snapshots and emits the filenames which could + * be removed in the {@link #DELETE_STREAM} side output. + */ +@Internal +public class ExpireSnapshotsProcessor extends ProcessFunction { + private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsProcessor.class); + public static final OutputTag DELETE_STREAM = + new OutputTag<>("expire-snapshots-file-deletes-stream", Types.STRING); + + private final TableLoader tableLoader; + private final Long maxSnapshotAgeMs; + private final Integer numSnapshots; + private final Integer plannerPoolSize; + private transient ExecutorService plannerPool; + private transient Table table; + + public ExpireSnapshotsProcessor( + TableLoader tableLoader, + Long maxSnapshotAgeMs, + Integer numSnapshots, + Integer plannerPoolSize) { + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + + this.tableLoader = tableLoader; + this.maxSnapshotAgeMs = maxSnapshotAgeMs; + this.numSnapshots = numSnapshots; + this.plannerPoolSize = plannerPoolSize; + } + + @Override + public void open(Configuration parameters) throws Exception { + tableLoader.open(); + this.table = tableLoader.loadTable(); + this.plannerPool = + plannerPoolSize != null + ? ThreadPools.newWorkerPool(table.name() + "-table--planner", plannerPoolSize) + : ThreadPools.getWorkerPool(); + } + + @Override + public void processElement(Trigger trigger, Context ctx, Collector out) + throws Exception { + try { + table.refresh(); + ExpireSnapshots expireSnapshots = table.expireSnapshots(); + if (maxSnapshotAgeMs != null) { + expireSnapshots = expireSnapshots.expireOlderThan(ctx.timestamp() - maxSnapshotAgeMs); + } + + if (numSnapshots != null) { + expireSnapshots = expireSnapshots.retainLast(numSnapshots); + } + + AtomicLong deleteFileCounter = new AtomicLong(0L); + expireSnapshots + .planWith(plannerPool) + .deleteWith( + file -> { + ctx.output(DELETE_STREAM, file); + deleteFileCounter.incrementAndGet(); + }) + .cleanExpiredFiles(true) + .commit(); + + LOG.info( + "Successfully finished expiring snapshots for {} at {}. Scheduled {} files for delete.", + table, + ctx.timestamp(), + deleteFileCounter.get()); + out.collect( + new TaskResult(trigger.taskId(), trigger.timestamp(), true, Collections.emptyList())); + } catch (Exception e) { + LOG.error("Failed to expiring snapshots for {} at {}", table, ctx.timestamp(), e); + out.collect( + new TaskResult(trigger.taskId(), trigger.timestamp(), false, Lists.newArrayList(e))); + } + } + + @Override + public void close() throws Exception { + super.close(); + + tableLoader.close(); + if (plannerPoolSize != null) { + plannerPool.shutdown(); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java index 3c3761ef2f4d..14d590162c8b 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java @@ -26,6 +26,9 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; @@ -64,6 +67,7 @@ public class LockRemover extends AbstractStreamOperator implements OneInputStreamOperator { private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + private final String tableName; private final TriggerLockFactory lockFactory; private final List maintenanceTaskNames; @@ -74,12 +78,14 @@ public class LockRemover extends AbstractStreamOperator private transient TriggerLockFactory.Lock recoveryLock; private transient long lastProcessedTaskStartEpoch = 0L; - public LockRemover(TriggerLockFactory lockFactory, List maintenanceTaskNames) { + public LockRemover( + String tableName, TriggerLockFactory lockFactory, List maintenanceTaskNames) { Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); Preconditions.checkArgument( maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(), "Invalid maintenance task names: null or empty"); + this.tableName = tableName; this.lockFactory = lockFactory; this.maintenanceTaskNames = maintenanceTaskNames; } @@ -91,22 +97,31 @@ public void open() throws Exception { Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); this.failedTaskResultCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); this.taskLastRunDurationMs = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); - for (String name : maintenanceTaskNames) { + for (int taskIndex = 0; taskIndex < maintenanceTaskNames.size(); ++taskIndex) { succeededTaskResultCounters.add( getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) .counter(TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER)); failedTaskResultCounters.add( getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) .counter(TableMaintenanceMetrics.FAILED_TASK_COUNTER)); AtomicLong duration = new AtomicLong(0); taskLastRunDurationMs.add(duration); getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) .gauge(TableMaintenanceMetrics.LAST_RUN_DURATION_MS, duration::get); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java index 89efffa15f16..d74b2349b1de 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java @@ -43,7 +43,7 @@ /** Monitors an Iceberg table for changes */ @Internal -class MonitorSource extends SingleThreadedIteratorSource { +public class MonitorSource extends SingleThreadedIteratorSource { private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final TableLoader tableLoader; @@ -58,7 +58,7 @@ class MonitorSource extends SingleThreadedIteratorSource { * @param rateLimiterStrategy limits the frequency the table is checked * @param maxReadBack sets the number of snapshots read before stopping change collection */ - MonitorSource( + public MonitorSource( TableLoader tableLoader, RateLimiterStrategy rateLimiterStrategy, long maxReadBack) { Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); Preconditions.checkNotNull(rateLimiterStrategy, "Rate limiter strategy should no be null"); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java index 773b34b6c495..78be0a9c15d6 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -28,7 +28,7 @@ /** Event describing changes in an Iceberg table */ @Internal -class TableChange { +public class TableChange { private int dataFileCount; private long dataFileSizeInBytes; private int posDeleteFileCount; @@ -37,7 +37,7 @@ class TableChange { private long eqDeleteRecordCount; private int commitCount; - TableChange( + private TableChange( int dataFileCount, long dataFileSizeInBytes, int posDeleteFileCount, @@ -87,7 +87,7 @@ static TableChange empty() { return new TableChange(0, 0L, 0, 0L, 0, 0L, 0); } - static Builder builder() { + public static Builder builder() { return new Builder(); } @@ -115,7 +115,7 @@ long eqDeleteRecordCount() { return eqDeleteRecordCount; } - public int commitCount() { + int commitCount() { return commitCount; } @@ -183,7 +183,7 @@ public int hashCode() { commitCount); } - static class Builder { + public static class Builder { private int dataFileCount = 0; private long dataFileSizeInBytes = 0L; private int posDeleteFileCount = 0; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java index 1a04461aed43..6147c3a5fd16 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java @@ -19,8 +19,10 @@ package org.apache.iceberg.flink.maintenance.operator; public class TableMaintenanceMetrics { - public static final String GROUP_KEY = "maintenanceTask"; - public static final String GROUP_VALUE_DEFAULT = "maintenanceTask"; + public static final String GROUP_KEY = "maintenance"; + public static final String TASK_NAME_KEY = "taskName"; + public static final String TASK_INDEX_KEY = "taskIndex"; + public static final String TABLE_NAME_KEY = "tableName"; // TriggerManager metrics public static final String RATE_LIMITER_TRIGGERED = "rateLimiterTriggered"; @@ -33,6 +35,10 @@ public class TableMaintenanceMetrics { public static final String FAILED_TASK_COUNTER = "failedTasks"; public static final String LAST_RUN_DURATION_MS = "lastRunDurationMs"; + // DeleteFiles metrics + public static final String DELETE_FILE_FAILED_COUNTER = "deleteFailed"; + public static final String DELETE_FILE_SUCCEEDED_COUNTER = "deleteSucceeded"; + private TableMaintenanceMetrics() { // do not instantiate } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java index dba33b22a42a..d448898bdfe6 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java @@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory; @Internal -class TriggerEvaluator implements Serializable { +public class TriggerEvaluator implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(TriggerEvaluator.class); private final List predicates; @@ -50,7 +50,7 @@ boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { return result; } - static class Builder implements Serializable { + public static class Builder implements Serializable { private Integer dataFileCount; private Long dataFileSizeInBytes; private Integer posDeleteFileCount; @@ -95,12 +95,12 @@ public Builder commitCount(int newCommitCount) { return this; } - Builder timeout(Duration newTimeout) { + public Builder timeout(Duration newTimeout) { this.timeout = newTimeout; return this; } - TriggerEvaluator build() { + public TriggerEvaluator build() { List predicates = Lists.newArrayList(); if (dataFileCount != null) { predicates.add((change, unused, unused2) -> change.dataFileCount() >= dataFileCount); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java index dc95b27af0a6..a96e99d94299 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.List; -import java.util.stream.Collectors; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -36,8 +35,9 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.util.Collector; -import org.apache.iceberg.SerializableTable; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; @@ -57,11 +57,11 @@ * the timer functions are available, but the key is not used. */ @Internal -class TriggerManager extends KeyedProcessFunction +public class TriggerManager extends KeyedProcessFunction implements CheckpointedFunction { private static final Logger LOG = LoggerFactory.getLogger(TriggerManager.class); - private final TableLoader tableLoader; + private final String tableName; private final TriggerLockFactory lockFactory; private final List maintenanceTaskNames; private final List evaluators; @@ -89,7 +89,7 @@ class TriggerManager extends KeyedProcessFunction private transient int startsFrom = 0; private transient boolean triggered = false; - TriggerManager( + public TriggerManager( TableLoader tableLoader, TriggerLockFactory lockFactory, List maintenanceTaskNames, @@ -110,7 +110,8 @@ class TriggerManager extends KeyedProcessFunction Preconditions.checkArgument( lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); - this.tableLoader = tableLoader; + tableLoader.open(); + this.tableName = tableLoader.loadTable().name(); this.lockFactory = lockFactory; this.maintenanceTaskNames = maintenanceTaskNames; this.evaluators = evaluators; @@ -123,30 +124,32 @@ public void open(Configuration parameters) throws Exception { this.rateLimiterTriggeredCounter = getRuntimeContext() .getMetricGroup() - .addGroup( - TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); this.concurrentRunThrottledCounter = getRuntimeContext() .getMetricGroup() - .addGroup( - TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) .counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED); this.nothingToTriggerCounter = getRuntimeContext() .getMetricGroup() - .addGroup( - TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); - this.triggerCounters = - maintenanceTaskNames.stream() - .map( - name -> - getRuntimeContext() - .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) - .counter(TableMaintenanceMetrics.TRIGGERED)) - .collect(Collectors.toList()); + this.triggerCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + for (int taskIndex = 0; taskIndex < maintenanceTaskNames.size(); ++taskIndex) { + triggerCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) + .counter(TableMaintenanceMetrics.TRIGGERED)); + } this.nextEvaluationTimeState = getRuntimeContext() @@ -159,8 +162,6 @@ public void open(Configuration parameters) throws Exception { this.lastTriggerTimesState = getRuntimeContext() .getListState(new ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG)); - - tableLoader.open(); } @Override @@ -220,7 +221,6 @@ public void onTimer(long timestamp, OnTimerContext ctx, Collector out) @Override public void close() throws IOException { - tableLoader.close(); lockFactory.close(); } @@ -256,10 +256,8 @@ private void checkAndFire(long current, TimerService timerService, Collector + *

  • {@link StreamExecutionEnvironment} - environment for testing + *
  • {@link ManualSource} - source for manually emitting {@link Trigger}s + *
  • {@link DataStream} - which generated from the {@link ManualSource} + *
  • {@link CollectingSink} - which could be used poll for the records emitted by the + * maintenance tasks + * + */ +class MaintenanceTaskInfraExtension implements BeforeEachCallback { + private StreamExecutionEnvironment env; + private ManualSource source; + private DataStream triggerStream; + private CollectingSink sink; + + @Override + public void beforeEach(ExtensionContext context) { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + source = new ManualSource<>(env, TypeInformation.of(Trigger.class)); + // Adds the watermark to mimic the behaviour expected for the input of the maintenance tasks + triggerStream = + source + .dataStream() + .assignTimestampsAndWatermarks(new TableMaintenance.PunctuatedWatermarkStrategy()) + .name(IGNORED_OPERATOR_NAME) + .forceNonParallel(); + sink = new CollectingSink<>(); + } + + StreamExecutionEnvironment env() { + return env; + } + + ManualSource source() { + return source; + } + + DataStream triggerStream() { + return triggerStream; + } + + CollectingSink sink() { + return sink; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java new file mode 100644 index 000000000000..36041d9c3831 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java @@ -0,0 +1,64 @@ +/* + * 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.flink.maintenance.api; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.function.Supplier; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.flink.maintenance.operator.CollectingSink; +import org.apache.iceberg.flink.maintenance.operator.ManualSource; +import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.extension.RegisterExtension; + +class MaintenanceTaskTestBase extends OperatorTestBase { + private static final int TESTING_TASK_ID = 0; + private static final Duration POLL_DURATION = Duration.ofSeconds(5); + + @RegisterExtension MaintenanceTaskInfraExtension infra = new MaintenanceTaskInfraExtension(); + + void runAndWaitForSuccess( + StreamExecutionEnvironment env, + ManualSource triggerSource, + CollectingSink collectingSink, + Supplier waitForCondition) + throws Exception { + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Do a single task run + long time = System.currentTimeMillis(); + triggerSource.sendRecord(Trigger.create(time, TESTING_TASK_ID), time); + + TaskResult result = collectingSink.poll(POLL_DURATION); + + assertThat(result.startEpoch()).isEqualTo(time); + assertThat(result.success()).isTrue(); + assertThat(result.taskIndex()).isEqualTo(TESTING_TASK_ID); + + Awaitility.await().until(waitForCondition::get); + } finally { + closeJobClient(jobClient); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java new file mode 100644 index 000000000000..f80129f966e1 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java @@ -0,0 +1,254 @@ +/* + * 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.flink.maintenance.api; + +import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; +import static org.apache.iceberg.flink.maintenance.api.ExpireSnapshots.DELETE_FILES_OPERATOR_NAME; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.List; +import java.util.Set; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestExpireSnapshots extends MaintenanceTaskTestBase { + private Table table; + + @BeforeEach + void before() { + MetricsReporterFactoryForTests.reset(); + this.table = createTable(); + tableLoader().open(); + } + + @Test + void testExpireSnapshots() throws Exception { + insert(table, 1, "a"); + insert(table, 2, "b"); + insert(table, 3, "c"); + insert(table, 4, "d"); + + Set snapshots = Sets.newHashSet(table.snapshots()); + assertThat(snapshots).hasSize(4); + + ExpireSnapshots.builder() + .parallelism(1) + .planningWorkerPoolSize(2) + .deleteBatchSize(3) + .maxSnapshotAge(Duration.ZERO) + .retainLast(1) + .uidSuffix(UID_SUFFIX) + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + "OTHER", + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + runAndWaitForSuccess( + infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(table.name(), 3L)); + + table.refresh(); + assertThat(Sets.newHashSet(table.snapshots())).hasSize(1); + // Check that the table data not changed + SimpleDataUtil.assertTableRecords( + table, + ImmutableList.of( + createRecord(1, "a"), + createRecord(2, "b"), + createRecord(3, "c"), + createRecord(4, "d"))); + } + + @Test + void testFailure() throws Exception { + insert(table, 1, "a"); + insert(table, 2, "b"); + + ExpireSnapshots.builder() + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + UID_SUFFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + JobClient jobClient = null; + try { + jobClient = infra.env().executeAsync(); + + // Do a single task run + long time = System.currentTimeMillis(); + infra.source().sendRecord(Trigger.create(time, 1), time); + + // First successful run (ensure that the operators are loaded/opened etc.) + assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isTrue(); + + // Drop the table, so it will cause an exception + dropTable(); + + // Failed run + infra.source().sendRecord(Trigger.create(time + 1, 1), time + 1); + + assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isFalse(); + } finally { + closeJobClient(jobClient); + } + + // Check the metrics. There are no expired snapshots or data files because ExpireSnapshots has + // no max age of number of snapshots set, so no files are removed. + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder, Long>() + .put( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_FAILED_COUNTER), + 0L) + .put( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_SUCCEEDED_COUNTER), + 0L) + .build()); + } + + @Test + void testUidAndSlotSharingGroup() { + ExpireSnapshots.builder() + .slotSharingGroup(SLOT_SHARING_GROUP) + .uidSuffix(UID_SUFFIX) + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + UID_SUFFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + checkUidsAreSet(infra.env(), UID_SUFFIX); + checkSlotSharingGroupsAreSet(infra.env(), SLOT_SHARING_GROUP); + } + + @Test + void testUidAndSlotSharingGroupUnset() { + ExpireSnapshots.builder() + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + UID_SUFFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + checkUidsAreSet(infra.env(), null); + checkSlotSharingGroupsAreSet(infra.env(), null); + } + + @Test + void testMetrics() throws Exception { + insert(table, 1, "a"); + insert(table, 2, "b"); + + ExpireSnapshots.builder() + .maxSnapshotAge(Duration.ZERO) + .retainLast(1) + .parallelism(1) + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + UID_SUFFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + runAndWaitForSuccess( + infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(table.name(), 1L)); + + // Check the metrics + Awaitility.await() + .untilAsserted( + () -> + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder, Long>() + .put( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_FAILED_COUNTER), + 0L) + .put( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_SUCCEEDED_COUNTER), + 1L) + .build())); + } + + private static boolean checkDeleteFinished(String tableName, Long expectedDeleteNum) { + return expectedDeleteNum.equals( + MetricsReporterFactoryForTests.counter( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + tableName, + DUMMY_TASK_NAME, + "0", + DELETE_FILE_SUCCEEDED_COUNTER))); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java similarity index 92% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java index 051d09d92bad..c8fa2a7d362a 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java @@ -16,9 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; -import static org.apache.iceberg.flink.maintenance.operator.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; +import static org.apache.iceberg.flink.maintenance.api.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java similarity index 97% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java index bf9e86f2534d..c06bef9087d5 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import static org.assertj.core.api.Assertions.assertThat; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java new file mode 100644 index 000000000000..467ad2d8ced9 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java @@ -0,0 +1,67 @@ +/* + * 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.flink.maintenance.api; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.time.Duration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestMaintenanceE2E extends OperatorTestBase { + private StreamExecutionEnvironment env; + + @BeforeEach + public void beforeEach() throws IOException { + this.env = StreamExecutionEnvironment.getExecutionEnvironment(); + Table table = createTable(); + insert(table, 1, "a"); + } + + @Test + void testE2e() throws Exception { + TableMaintenance.forTable(env, tableLoader(), LOCK_FACTORY) + .uidSuffix("E2eTestUID") + .rateLimit(Duration.ofMinutes(10)) + .lockCheckDelay(Duration.ofSeconds(10)) + .add( + ExpireSnapshots.builder() + .scheduleOnCommitCount(10) + .maxSnapshotAge(Duration.ofMinutes(10)) + .retainLast(5) + .deleteBatchSize(5) + .parallelism(8)) + .append(); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Just make sure that we are able to instantiate the flow + assertThat(jobClient).isNotNull(); + } finally { + closeJobClient(jobClient); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java new file mode 100644 index 000000000000..0e4a72bd16f8 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java @@ -0,0 +1,460 @@ +/* + * 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.flink.maintenance.api; + +import static org.apache.iceberg.flink.SimpleDataUtil.createRowData; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.LOCK_REMOVER_OPERATOR_NAME; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.SOURCE_OPERATOR_NAME_PREFIX; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.TRIGGER_MANAGER_OPERATOR_NAME; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.transformations.SourceTransformation; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.ManualSource; +import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests; +import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; +import org.apache.iceberg.flink.maintenance.operator.TableChange; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +class TestTableMaintenance extends OperatorTestBase { + private static final String[] TASKS = + new String[] { + MaintenanceTaskBuilderForTest.class.getSimpleName() + " [0]", + MaintenanceTaskBuilderForTest.class.getSimpleName() + " [1]" + }; + private static final TableChange DUMMY_CHANGE = TableChange.builder().commitCount(1).build(); + private static final List PROCESSED = + Collections.synchronizedList(Lists.newArrayListWithCapacity(1)); + + private StreamExecutionEnvironment env; + private Table table; + + @TempDir private File checkpointDir; + + @BeforeEach + public void beforeEach() throws IOException { + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + this.env = StreamExecutionEnvironment.getExecutionEnvironment(config); + this.table = createTable(); + insert(table, 1, "a"); + + PROCESSED.clear(); + MaintenanceTaskBuilderForTest.counter = 0; + } + + @Test + void testForChangeStream() throws Exception { + ManualSource schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader(), LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .lockCheckDelay(Duration.ofSeconds(3)) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .scheduleOnDataFileCount(2) + .scheduleOnDataFileSize(3L) + .scheduleOnEqDeleteFileCount(4) + .scheduleOnEqDeleteRecordCount(5L) + .scheduleOnPosDeleteFileCount(6) + .scheduleOnPosDeleteRecordCount(7L) + .scheduleOnInterval(Duration.ofHours(1))); + + sendEvents(schedulerSource, streamBuilder, ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1))); + } + + @Test + void testForTable() throws Exception { + TableLoader tableLoader = tableLoader(); + + env.enableCheckpointing(10); + + TableMaintenance.forTable(env, tableLoader, LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .maxReadBack(2) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(2)) + .append(); + + // Creating a stream for inserting data into the table concurrently + ManualSource insertSource = + new ManualSource<>(env, InternalTypeInfo.of(FlinkSchemaUtil.convert(table.schema()))); + FlinkSink.forRowData(insertSource.dataStream()) + .tableLoader(tableLoader) + .uidPrefix(UID_SUFFIX + "-iceberg-sink") + .append(); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + insertSource.sendRecord(createRowData(2, "b")); + + Awaitility.await().until(() -> PROCESSED.size() == 1); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testLocking() throws Exception { + TriggerLockFactory.Lock lock = LOCK_FACTORY.createLock(); + + ManualSource schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader(), LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)); + + assertThat(lock.isHeld()).isFalse(); + sendEvents(schedulerSource, streamBuilder, ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1))); + + assertThat(lock.isHeld()).isFalse(); + } + + @Test + void testMetrics() throws Exception { + ManualSource schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader(), LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .lockCheckDelay(Duration.ofMillis(2)) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) + .add(new MaintenanceTaskBuilderForTest(false).scheduleOnCommitCount(2)); + + sendEvents( + schedulerSource, + streamBuilder, + ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1), Tuple2.of(DUMMY_CHANGE, 2))); + + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, + table.name(), + TASKS[0], + "0", + SUCCEEDED_TASK_COUNTER)) + .equals(2L)); + + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder, Long>() + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, + table.name(), + TASKS[0], + "0", + SUCCEEDED_TASK_COUNTER), + 2L) + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, table.name(), TASKS[0], "0", FAILED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), TASKS[0], "0", TRIGGERED), + 2L) + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, + table.name(), + TASKS[1], + "1", + SUCCEEDED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, table.name(), TASKS[1], "1", FAILED_TASK_COUNTER), + 1L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), TASKS[1], "1", TRIGGERED), + 1L) + .put( + ImmutableList.of(TRIGGER_MANAGER_OPERATOR_NAME, table.name(), NOTHING_TO_TRIGGER), + -1L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), CONCURRENT_RUN_THROTTLED), + -1L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), RATE_LIMITER_TRIGGERED), + -1L) + .build()); + } + + @Test + void testUidAndSlotSharingGroup() throws IOException { + TableMaintenance.forChangeStream( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader(), + LOCK_FACTORY) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP)) + .append(); + + checkUidsAreSet(env, UID_SUFFIX); + checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); + } + + @Test + void testUidAndSlotSharingGroupUnset() throws IOException { + TableMaintenance.forChangeStream( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader(), + LOCK_FACTORY) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) + .append(); + + checkUidsAreSet(env, null); + checkSlotSharingGroupsAreSet(env, null); + } + + @Test + void testUidAndSlotSharingGroupInherit() throws IOException { + TableMaintenance.forChangeStream( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader(), + LOCK_FACTORY) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) + .append(); + + checkUidsAreSet(env, UID_SUFFIX); + checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); + } + + @Test + void testUidAndSlotSharingGroupOverWrite() throws IOException { + String anotherUid = "Another-UID"; + String anotherSlotSharingGroup = "Another-SlotSharingGroup"; + TableMaintenance.forChangeStream( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader(), + LOCK_FACTORY) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .uidSuffix(anotherUid) + .slotSharingGroup(anotherSlotSharingGroup)) + .append(); + + // Choose an operator from the scheduler part of the graph + Transformation schedulerTransformation = + env.getTransformations().stream() + .filter(t -> t.getName().equals("Trigger manager")) + .findFirst() + .orElseThrow(); + assertThat(schedulerTransformation.getUid()).contains(UID_SUFFIX); + assertThat(schedulerTransformation.getSlotSharingGroup()).isPresent(); + assertThat(schedulerTransformation.getSlotSharingGroup().get().getName()) + .isEqualTo(SLOT_SHARING_GROUP); + + // Choose an operator from the maintenance task part of the graph + Transformation scheduledTransformation = + env.getTransformations().stream() + .filter( + t -> t.getName().startsWith(MaintenanceTaskBuilderForTest.class.getSimpleName())) + .findFirst() + .orElseThrow(); + assertThat(scheduledTransformation.getUid()).contains(anotherUid); + assertThat(scheduledTransformation.getSlotSharingGroup()).isPresent(); + assertThat(scheduledTransformation.getSlotSharingGroup().get().getName()) + .isEqualTo(anotherSlotSharingGroup); + } + + @Test + void testUidAndSlotSharingGroupForMonitorSource() throws IOException { + TableMaintenance.forTable(env, tableLoader(), LOCK_FACTORY) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP)) + .append(); + + Transformation source = monitorSource(); + assertThat(source).isNotNull(); + assertThat(source.getUid()).contains(UID_SUFFIX); + assertThat(source.getSlotSharingGroup()).isPresent(); + assertThat(source.getSlotSharingGroup().get().getName()).isEqualTo(SLOT_SHARING_GROUP); + + checkUidsAreSet(env, UID_SUFFIX); + checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); + } + + /** + * Sends the events though the {@link ManualSource} provided, and waits until the given number of + * records are processed. + * + * @param schedulerSource used for sending the events + * @param streamBuilder used for generating the job + * @param eventsAndResultNumbers the pair of the event and the expected processed records + * @throws Exception if any + */ + private void sendEvents( + ManualSource schedulerSource, + TableMaintenance.Builder streamBuilder, + List> eventsAndResultNumbers) + throws Exception { + streamBuilder.append(); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + eventsAndResultNumbers.forEach( + eventsAndResultNumber -> { + int expectedSize = PROCESSED.size() + eventsAndResultNumber.f1; + schedulerSource.sendRecord(eventsAndResultNumber.f0); + Awaitility.await() + .until( + () -> PROCESSED.size() == expectedSize && !LOCK_FACTORY.createLock().isHeld()); + }); + } finally { + closeJobClient(jobClient); + } + } + + /** + * Finds the {@link org.apache.iceberg.flink.maintenance.operator.MonitorSource} for testing + * purposes by parsing the transformation tree. + * + * @return The monitor source if we found it + */ + private Transformation monitorSource() { + assertThat(env.getTransformations()).isNotEmpty(); + assertThat(env.getTransformations().get(0).getInputs()).isNotEmpty(); + assertThat(env.getTransformations().get(0).getInputs().get(0).getInputs()).isNotEmpty(); + + Transformation result = + env.getTransformations().get(0).getInputs().get(0).getInputs().get(0); + + // Some checks to make sure this is the transformation we are looking for + assertThat(result).isInstanceOf(SourceTransformation.class); + assertThat(result.getName()).startsWith(SOURCE_OPERATOR_NAME_PREFIX); + + return result; + } + + private static class MaintenanceTaskBuilderForTest + extends MaintenanceTaskBuilder { + private final boolean success; + private final int id; + private static int counter = 0; + + MaintenanceTaskBuilderForTest(boolean success) { + this.success = success; + this.id = counter; + ++counter; + } + + @Override + DataStream append(DataStream trigger) { + String name = TASKS[id]; + return trigger + .map(new DummyMaintenanceTask(success)) + .name(name) + .uid(uidSuffix() + "-test-mapper-" + name + "-" + id) + .slotSharingGroup(slotSharingGroup()) + .forceNonParallel(); + } + } + + private static class DummyMaintenanceTask + implements MapFunction, ResultTypeQueryable, Serializable { + private final boolean success; + + private DummyMaintenanceTask(boolean success) { + this.success = success; + } + + @Override + public TaskResult map(Trigger trigger) { + // Ensure that the lock is held when processing + assertThat(LOCK_FACTORY.createLock().isHeld()).isTrue(); + PROCESSED.add(trigger); + + return new TaskResult( + trigger.taskId(), + trigger.timestamp(), + success, + success ? Collections.emptyList() : Lists.newArrayList(new Exception("Testing error"))); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(TaskResult.class); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java index 9b6580fad0bf..e7e818ba6887 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java @@ -31,7 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; /** Sink for collecting output during testing. */ -class CollectingSink implements Sink { +public class CollectingSink implements Sink { private static final long serialVersionUID = 1L; private static final List> QUEUES = Collections.synchronizedList(Lists.newArrayListWithExpectedSize(1)); @@ -39,7 +39,7 @@ class CollectingSink implements Sink { private final int index; /** Creates a new sink which collects the elements received. */ - CollectingSink() { + public CollectingSink() { this.index = NUM_SINKS.incrementAndGet(); QUEUES.add(new LinkedBlockingQueue<>()); } @@ -69,7 +69,7 @@ boolean isEmpty() { * @return The first element received by this {@link Sink} * @throws TimeoutException if no element received until the timeout */ - T poll(Duration timeout) throws TimeoutException { + public T poll(Duration timeout) throws TimeoutException { Object element; try { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java deleted file mode 100644 index 36e162d4f068..000000000000 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -class ConstantsForTests { - public static final long EVENT_TIME = 10L; - static final long EVENT_TIME_2 = 11L; - static final String DUMMY_NAME = "dummy"; - - private ConstantsForTests() { - // Do not instantiate - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java deleted file mode 100644 index 91d36aa3e85d..000000000000 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.jupiter.api.extension.AfterEachCallback; -import org.junit.jupiter.api.extension.BeforeEachCallback; -import org.junit.jupiter.api.extension.ExtensionContext; - -/** - * Junit 5 extension for running Flink SQL queries. {@link - * org.apache.flink.test.junit5.MiniClusterExtension} is used for executing the SQL batch jobs. - */ -public class FlinkSqlExtension implements BeforeEachCallback, AfterEachCallback { - private final String catalogName; - private final Map catalogProperties; - private final String databaseName; - private final Path warehouse; - private final CatalogLoader catalogLoader; - private TableEnvironment tableEnvironment; - - public FlinkSqlExtension( - String catalogName, Map catalogProperties, String databaseName) { - this.catalogName = catalogName; - this.catalogProperties = Maps.newHashMap(catalogProperties); - this.databaseName = databaseName; - - // Add temporary dir as a warehouse location - try { - this.warehouse = Files.createTempDirectory("warehouse"); - } catch (IOException e) { - throw new RuntimeException(e); - } - this.catalogProperties.put( - CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouse)); - this.catalogLoader = - CatalogLoader.hadoop(catalogName, new Configuration(), this.catalogProperties); - } - - @Override - public void beforeEach(ExtensionContext context) { - // We need to recreate the tableEnvironment for every test as the minicluster is recreated - this.tableEnvironment = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - exec("CREATE CATALOG %s WITH %s", catalogName, toWithClause(catalogProperties)); - exec("CREATE DATABASE %s.%s", catalogName, databaseName); - exec("USE CATALOG %s", catalogName); - exec("USE %s", databaseName); - } - - @Override - public void afterEach(ExtensionContext context) throws IOException { - List tables = exec("SHOW TABLES"); - tables.forEach(t -> exec("DROP TABLE IF EXISTS %s", t.getField(0))); - exec("USE CATALOG default_catalog"); - exec("DROP CATALOG IF EXISTS %s", catalogName); - try (Stream files = Files.walk(warehouse)) { - files.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); - } - } - - /** - * Executes an SQL query with the given parameters. The parameter substitution is done by {@link - * String#format(String, Object...)}. - * - * @param query to run - * @param parameters to substitute to the query - * @return The {@link Row}s returned by the query - */ - public List exec(String query, Object... parameters) { - TableResult tableResult = tableEnvironment.executeSql(String.format(query, parameters)); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - /** - * Returns the {@link TableLoader} which could be used to access the given table. - * - * @param tableName of the table - * @return the {@link TableLoader} for the table - */ - public TableLoader tableLoader(String tableName) { - TableLoader tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(databaseName, tableName)); - tableLoader.open(); - return tableLoader; - } - - private static String toWithClause(Map props) { - return String.format( - "(%s)", - props.entrySet().stream() - .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) - .collect(Collectors.joining(","))); - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java deleted file mode 100644 index 9cdc55cb0cce..000000000000 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -import java.io.File; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; -import org.awaitility.Awaitility; - -class FlinkStreamingTestUtils { - private FlinkStreamingTestUtils() { - // Do not instantiate - } - - /** - * Close the {@link JobClient} and wait for the job closure. If the savepointDir is specified, it - * stops the job with a savepoint. - * - * @param jobClient the job to close - * @param savepointDir the savepointDir to store the last savepoint. If null then - * stop without a savepoint. - * @return configuration for restarting the job from the savepoint - */ - static Configuration closeJobClient(JobClient jobClient, File savepointDir) { - Configuration conf = new Configuration(); - if (jobClient != null) { - if (savepointDir != null) { - // Stop with savepoint - jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); - // Wait until the savepoint is created and the job has been stopped - Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); - conf.set( - SavepointConfigOptions.SAVEPOINT_PATH, - savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath()); - } else { - jobClient.cancel(); - } - - // Wait until the job has been stopped - Awaitility.await().until(() -> jobClient.getJobStatus().get().isTerminalState()); - return conf; - } - - return null; - } - - /** - * Close the {@link JobClient} and wait for the job closure. - * - * @param jobClient the job to close - */ - static void closeJobClient(JobClient jobClient) { - closeJobClient(jobClient, null); - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java index 679b3ec508a2..eff32fcfa118 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java @@ -44,7 +44,7 @@ import org.jetbrains.annotations.Nullable; /** Testing source implementation for Flink sources which can be triggered manually. */ -class ManualSource +public class ManualSource implements Source, ResultTypeQueryable { @@ -65,7 +65,7 @@ class ManualSource * @param env to register the source * @param type of the events returned by the source */ - ManualSource(StreamExecutionEnvironment env, TypeInformation type) { + public ManualSource(StreamExecutionEnvironment env, TypeInformation type) { this.type = type; this.env = env; this.index = numSources++; @@ -78,7 +78,7 @@ class ManualSource * * @param event to emit */ - void sendRecord(T event) { + public void sendRecord(T event) { this.sendInternal(Tuple2.of(event, null)); } @@ -88,7 +88,7 @@ void sendRecord(T event) { * @param event to emit * @param eventTime of the event */ - void sendRecord(T event, long eventTime) { + public void sendRecord(T event, long eventTime) { this.sendInternal(Tuple2.of(event, eventTime)); } @@ -97,7 +97,7 @@ void sendRecord(T event, long eventTime) { * * @param timeStamp of the watermark */ - void sendWatermark(long timeStamp) { + public void sendWatermark(long timeStamp) { this.sendInternal(Tuple2.of(null, timeStamp)); } @@ -112,7 +112,7 @@ void markFinished() { * * @return the stream emitted by this source */ - DataStream dataStream() { + public DataStream dataStream() { if (this.stream == null) { this.stream = this.env diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java index 7a523035b7fb..ed66ff3df076 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; @@ -38,10 +39,24 @@ public class MetricsReporterFactoryForTests implements MetricReporterFactory { private static final TestMetricsReporter INSTANCE = new TestMetricsReporter(); - private static final Pattern FULL_METRIC_NAME = + private static final Pattern TASK_METRIC_NAME = Pattern.compile( "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + TableMaintenanceMetrics.GROUP_KEY + + "\\." + + TableMaintenanceMetrics.TABLE_NAME_KEY + + "\\.([^.]+)\\." + + TableMaintenanceMetrics.TASK_NAME_KEY + + "\\.([^.]+)\\." + + TableMaintenanceMetrics.TASK_INDEX_KEY + + "\\.([^.]+)\\.([^.]+)"); + + private static final Pattern MAIN_METRIC_NAME = + Pattern.compile( + "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + + TableMaintenanceMetrics.GROUP_KEY + + "\\." + + TableMaintenanceMetrics.TABLE_NAME_KEY + "\\.([^.]+)\\.([^.]+)"); private static Map counters = Maps.newConcurrentMap(); @@ -72,20 +87,26 @@ public static void reset() { gauges = Maps.newConcurrentMap(); } - public static Long counter(String name) { - return counterValues().get(name); + public static Long counter(List parts) { + return counterValues().get(longName(parts)); } - public static Long gauge(String name) { - return gaugeValues().get(name); + public static Long gauge(List parts) { + return gaugeValues().get(longName(parts)); } - public static void assertGauges(Map expected) { - assertThat(filter(gaugeValues(), expected)).isEqualTo(filter(expected, expected)); + public static void assertGauges(Map, Long> expected) { + Map transformed = + expected.entrySet().stream() + .collect(Collectors.toMap(k -> longName(k.getKey()), Map.Entry::getValue)); + assertThat(filter(gaugeValues(), transformed)).isEqualTo(filter(transformed, transformed)); } - public static void assertCounters(Map expected) { - assertThat(filter(counterValues(), expected)).isEqualTo(filter(expected, expected)); + public static void assertCounters(Map, Long> expected) { + Map transformed = + expected.entrySet().stream() + .collect(Collectors.toMap(k -> longName(k.getKey()), Map.Entry::getValue)); + assertThat(filter(counterValues(), transformed)).isEqualTo(filter(transformed, transformed)); } private static Map gaugeValues() { @@ -113,12 +134,30 @@ private static Map filter(Map original, Map parts) { + return parts.stream().map(s -> s.replaceAll("\\.", "_")).collect(Collectors.joining(".")); } private static class TestMetricsReporter implements MetricReporter { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 225853086545..5e05f40e53cf 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -19,27 +19,62 @@ package org.apache.iceberg.flink.maintenance.operator; import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; +import java.io.IOException; +import java.nio.file.Path; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.flink.streaming.api.transformations.SinkTransformation; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.iceberg.flink.FlinkCatalogFactory; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; -class OperatorTestBase { +public class OperatorTestBase { private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; - private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); - private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); + private static final Schema SCHEMA_WITH_PRIMARY_KEY = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())), + ImmutableMap.of(), + ImmutableSet.of(SimpleDataUtil.SCHEMA.columns().get(0).fieldId())); - static final String TABLE_NAME = "test_table"; + protected static final String UID_SUFFIX = "UID-Dummy"; + protected static final String SLOT_SHARING_GROUP = "SlotSharingGroup"; + protected static final TriggerLockFactory LOCK_FACTORY = new MemoryLockFactory(); + + public static final String IGNORED_OPERATOR_NAME = "Ignore"; + + static final long EVENT_TIME = 10L; + static final long EVENT_TIME_2 = 11L; + protected static final String DUMMY_TASK_NAME = "dummyTask"; + protected static final String DUMMY_TABLE_NAME = "dummyTable"; @RegisterExtension protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = @@ -50,43 +85,57 @@ class OperatorTestBase { .setConfiguration(config()) .build()); + @TempDir private Path warehouseDir; + @RegisterExtension - final FlinkSqlExtension sql = - new FlinkSqlExtension( - "catalog", - ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), - "db"); + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - private static Configuration config() { - Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); - MetricOptions.forReporter(config, "test_reporter") - .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); - return config; + @BeforeEach + void before() { + LOCK_FACTORY.open(); + MetricsReporterFactoryForTests.reset(); } - protected static TriggerLockFactory lockFactory() { - return new TriggerLockFactory() { - @Override - public void open() { - MAINTENANCE_LOCK.unlock(); - RECOVERY_LOCK.unlock(); - } + @AfterEach + void after() throws IOException { + LOCK_FACTORY.close(); + } - @Override - public Lock createLock() { - return MAINTENANCE_LOCK; - } + protected static Table createTable() { + return CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("flink.max-continuous-empty-commits", "100000")); + } - @Override - public Lock createRecoveryLock() { - return RECOVERY_LOCK; - } + protected static Table createTableWithDelete() { + return CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA_WITH_PRIMARY_KEY, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("format-version", "2", "write.upsert.enabled", "true")); + } - @Override - public void close() { - // do nothing - } - }; + protected void insert(Table table, Integer id, String data) throws IOException { + new GenericAppenderHelper(table, FileFormat.PARQUET, warehouseDir) + .appendToTable(Lists.newArrayList(SimpleDataUtil.createRecord(id, data))); + table.refresh(); + } + + protected void dropTable() { + CATALOG_EXTENSION.catalogLoader().loadCatalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + protected TableLoader tableLoader() { + return CATALOG_EXTENSION.tableLoader(); } /** @@ -98,7 +147,7 @@ public void close() { * stop without a savepoint. * @return configuration for restarting the job from the savepoint */ - public static Configuration closeJobClient(JobClient jobClient, File savepointDir) { + protected static Configuration closeJobClient(JobClient jobClient, File savepointDir) { Configuration conf = new Configuration(); if (jobClient != null) { if (savepointDir != null) { @@ -126,12 +175,45 @@ public static Configuration closeJobClient(JobClient jobClient, File savepointDi * * @param jobClient the job to close */ - public static void closeJobClient(JobClient jobClient) { + protected static void closeJobClient(JobClient jobClient) { closeJobClient(jobClient, null); } + protected static void checkUidsAreSet(StreamExecutionEnvironment env, String uidSuffix) { + env.getTransformations().stream() + .filter( + t -> !(t instanceof SinkTransformation) && !(t.getName().equals(IGNORED_OPERATOR_NAME))) + .forEach( + transformation -> { + assertThat(transformation.getUid()).isNotNull(); + if (uidSuffix != null) { + assertThat(transformation.getUid()).contains(UID_SUFFIX); + } + }); + } + + protected static void checkSlotSharingGroupsAreSet(StreamExecutionEnvironment env, String name) { + String nameToCheck = name != null ? name : StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP; + + env.getTransformations().stream() + .filter( + t -> !(t instanceof SinkTransformation) && !(t.getName().equals(IGNORED_OPERATOR_NAME))) + .forEach( + t -> { + assertThat(t.getSlotSharingGroup()).isPresent(); + assertThat(t.getSlotSharingGroup().get().getName()).isEqualTo(nameToCheck); + }); + } + + private static Configuration config() { + Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + MetricOptions.forReporter(config, "test_reporter") + .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); + return config; + } + private static class MemoryLock implements TriggerLockFactory.Lock { - boolean locked = false; + volatile boolean locked = false; @Override public boolean tryLock() { @@ -153,4 +235,30 @@ public void unlock() { locked = false; } } + + private static class MemoryLockFactory implements TriggerLockFactory { + private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); + + @Override + public void open() { + MAINTENANCE_LOCK.unlock(); + RECOVERY_LOCK.unlock(); + } + + @Override + public Lock createLock() { + return MAINTENANCE_LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // do nothing + } + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java new file mode 100644 index 000000000000..d70c4aafd59a --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java @@ -0,0 +1,116 @@ +/* + * 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.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestDeleteFilesProcessor extends OperatorTestBase { + private static final String DUMMY_FILE_NAME = "dummy"; + private static final Set TABLE_FILES = + ImmutableSet.of( + "metadata/v1.metadata.json", + "metadata/version-hint.text", + "metadata/.version-hint.text.crc", + "metadata/.v1.metadata.json.crc"); + + private Table table; + + @BeforeEach + void before() { + this.table = createTable(); + } + + @Test + void testDelete() throws Exception { + // Write an extra file + Path dummyFile = Path.of(tablePath(table).toString(), DUMMY_FILE_NAME); + Files.write(dummyFile, "DUMMY".getBytes(StandardCharsets.UTF_8)); + + Set files = listFiles(table); + assertThat(files) + .containsAll(TABLE_FILES) + .contains(DUMMY_FILE_NAME) + .hasSize(TABLE_FILES.size() + 1); + + deleteFile(tableLoader(), dummyFile.toString()); + + assertThat(listFiles(table)).isEqualTo(TABLE_FILES); + } + + @Test + void testDeleteMissingFile() throws Exception { + Path dummyFile = + FileSystems.getDefault().getPath(table.location().substring(5), DUMMY_FILE_NAME); + + deleteFile(tableLoader(), dummyFile.toString()); + + assertThat(listFiles(table)).isEqualTo(TABLE_FILES); + } + + @Test + void testInvalidURIScheme() throws Exception { + deleteFile(tableLoader(), "wrong://"); + + assertThat(listFiles(table)).isEqualTo(TABLE_FILES); + } + + private void deleteFile(TableLoader tableLoader, String fileName) throws Exception { + tableLoader().open(); + try (OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>( + new DeleteFilesProcessor(0, DUMMY_TASK_NAME, tableLoader.loadTable(), 10), + StringSerializer.INSTANCE)) { + testHarness.open(); + testHarness.processElement(fileName, System.currentTimeMillis()); + testHarness.processWatermark(EVENT_TIME); + testHarness.endInput(); + } + } + + private static Path tablePath(Table table) { + return FileSystems.getDefault().getPath(table.location().substring(5)); + } + + private static Set listFiles(Table table) throws IOException { + String tableRootPath = TestFixtures.TABLE_IDENTIFIER.toString().replace(".", "/"); + return Files.find( + tablePath(table), Integer.MAX_VALUE, (filePath, fileAttr) -> fileAttr.isRegularFile()) + .map( + p -> + p.toString() + .substring(p.toString().indexOf(tableRootPath) + tableRootPath.length() + 1)) + .collect(Collectors.toSet()); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java new file mode 100644 index 000000000000..d312fc312c99 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java @@ -0,0 +1,80 @@ +/* + * 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.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Queue; +import java.util.Set; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +class TestExpireSnapshotsProcessor extends OperatorTestBase { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testExpire(boolean success) throws Exception { + Table table = createTable(); + insert(table, 1, "a"); + insert(table, 2, "b"); + + List actual; + Queue> deletes; + try (OneInputStreamOperatorTestHarness testHarness = + ProcessFunctionTestHarnesses.forProcessFunction( + new ExpireSnapshotsProcessor(tableLoader(), 0L, 1, 10))) { + testHarness.open(); + + if (!success) { + // Cause an exception + dropTable(); + } + + testHarness.processElement(Trigger.create(10, 11), System.currentTimeMillis()); + deletes = testHarness.getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM); + actual = testHarness.extractOutputValues(); + } + + assertThat(actual).hasSize(1); + TaskResult result = actual.get(0); + assertThat(result.startEpoch()).isEqualTo(10); + assertThat(result.taskIndex()).isEqualTo(11); + assertThat(result.success()).isEqualTo(success); + + if (success) { + assertThat(result.exceptions()).isNotNull().isEmpty(); + + table.refresh(); + Set snapshots = Sets.newHashSet(table.snapshots()); + assertThat(snapshots).hasSize(1); + assertThat(deletes).hasSize(1); + } else { + assertThat(result.exceptions()).isNotNull().hasSize(1); + assertThat(deletes).isNull(); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java index cffcc4eb0471..4e4de9e78e9e 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.LAST_RUN_DURATION_MS; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.util.Collection; +import java.util.List; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.sink2.Committer; @@ -43,6 +44,9 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; @@ -53,7 +57,7 @@ @Timeout(value = 10) class TestLockRemover extends OperatorTestBase { - private static final String[] TASKS = new String[] {"task0", "task1"}; + private static final String[] TASKS = new String[] {"task0", "task1", "task2"}; private static final TriggerLockFactory.Lock LOCK = new TestingLock(); private static final TriggerLockFactory.Lock RECOVERY_LOCK = new TestingLock(); @@ -71,9 +75,9 @@ void testProcess() throws Exception { source .dataStream() .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + new LockRemover(DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS))) .setParallelism(1); JobClient jobClient = null; @@ -128,37 +132,79 @@ void testMetrics() throws Exception { source .dataStream() .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + new LockRemover(DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS))) .setParallelism(1); JobClient jobClient = null; + long time = System.currentTimeMillis(); try { jobClient = env.executeAsync(); // Start the 2 successful and one failed result trigger for task1, and 3 successful for task2 - processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(1, 1L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(1, 2L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(0, 3L, false, Lists.newArrayList())); - processAndCheck(source, new TaskResult(0, 4L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(1, 5L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, time, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 0L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 0L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, time, false, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, time, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 0L, true, Lists.newArrayList())); Awaitility.await() .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER) + ImmutableList.of( + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + TASKS[1], + "1", + SUCCEEDED_TASK_COUNTER)) .equals(3L)); // Final check all the counters MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() - .put(DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER, 2L) - .put(DUMMY_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 1L) - .put(DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 3L) - .put(DUMMY_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 0L) + new ImmutableMap.Builder, Long>() + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[0], "0", SUCCEEDED_TASK_COUNTER), + 2L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[0], "0", FAILED_TASK_COUNTER), + 1L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[1], "1", SUCCEEDED_TASK_COUNTER), + 3L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[1], "1", FAILED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[2], "2", SUCCEEDED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[2], "2", FAILED_TASK_COUNTER), + 0L) .build()); + + assertThat( + MetricsReporterFactoryForTests.gauge( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[0], "0", LAST_RUN_DURATION_MS))) + .isPositive(); + assertThat( + MetricsReporterFactoryForTests.gauge( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[1], "1", LAST_RUN_DURATION_MS))) + .isGreaterThan(time); + assertThat( + MetricsReporterFactoryForTests.gauge( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[2], "2", LAST_RUN_DURATION_MS))) + .isZero(); } finally { closeJobClient(jobClient); } @@ -181,9 +227,10 @@ void testRecovery() throws Exception { .dataStream() .union(source2.dataStream()) .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) + new LockRemover( + DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) .setParallelism(1); JobClient jobClient = null; @@ -201,7 +248,12 @@ void testRecovery() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + ImmutableList.of( + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + TASKS[0], + "0", + SUCCEEDED_TASK_COUNTER)) .equals(2L)); // We did not remove the recovery lock, as no watermark received from the other source @@ -223,20 +275,21 @@ private void processAndCheck(ManualSource source, TaskResult input) private void processAndCheck( ManualSource source, TaskResult input, String counterPrefix) { + List counterKey = + ImmutableList.of( + (counterPrefix != null ? counterPrefix : "") + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + TASKS[input.taskIndex()], + String.valueOf(input.taskIndex()), + input.success() ? SUCCEEDED_TASK_COUNTER : FAILED_TASK_COUNTER); + Long counterValue = MetricsReporterFactoryForTests.counter(counterKey); + Long expected = counterValue != null ? counterValue + 1 : 1L; + source.sendRecord(input); source.sendWatermark(input.startEpoch()); - String counterName = - (counterPrefix != null ? counterPrefix : "") - .concat( - input.success() - ? DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + SUCCEEDED_TASK_COUNTER - : DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + FAILED_TASK_COUNTER); - Long counterValue = MetricsReporterFactoryForTests.counter(counterName); - Long expected = counterValue != null ? counterValue + 1 : 1L; - Awaitility.await() - .until(() -> expected.equals(MetricsReporterFactoryForTests.counter(counterName))); + .until(() -> expected.equals(MetricsReporterFactoryForTests.counter(counterKey))); } private static class TestingLockFactory implements TriggerLockFactory { @@ -370,9 +423,10 @@ public void flatMap( } }) .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); + new LockRemover( + DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); } } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index 3aee05322561..c561c7054eae 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; +import java.io.IOException; import java.time.Duration; import java.util.List; import java.util.concurrent.atomic.AtomicReference; @@ -60,40 +61,27 @@ class TestMonitorSource extends OperatorTestBase { @ParameterizedTest @ValueSource(booleans = {true, false}) - void testChangeReaderIterator(boolean withDelete) { - if (withDelete) { - sql.exec( - "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) WITH ('format-version'='2', 'write.upsert.enabled'='true')", - TABLE_NAME); - } else { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - } - - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); - Table table = tableLoader.loadTable(); + void testChangeReaderIterator(boolean withDelete) throws IOException { + Table table = withDelete ? createTableWithDelete() : createTable(); MonitorSource.TableChangeIterator iterator = - new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); + new MonitorSource.TableChangeIterator(tableLoader(), null, Long.MAX_VALUE); // For an empty table we get an empty result assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); // Add a single commit and get back the commit data in the event - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); - table.refresh(); + insert(table, 1, "a"); TableChange expected = tableChangeWithLastSnapshot(table, TableChange.empty()); assertThat(iterator.next()).isEqualTo(expected); // Make sure that consecutive calls do not return the data again assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); // Add two more commits, but fetch the data in one loop - sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); - table.refresh(); + insert(table, 2, "b"); expected = tableChangeWithLastSnapshot(table, TableChange.empty()); - sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); - table.refresh(); + insert(table, 3, "c"); expected = tableChangeWithLastSnapshot(table, expected); assertThat(iterator.next()).isEqualTo(expected); @@ -106,17 +94,11 @@ void testChangeReaderIterator(boolean withDelete) { */ @Test void testSource() throws Exception { - sql.exec( - "CREATE TABLE %s (id int, data varchar) " - + "WITH ('flink.max-continuous-empty-commits'='100000')", - TABLE_NAME); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); - Table table = tableLoader.loadTable(); + Table table = createTable(); DataStream events = env.fromSource( - new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + new MonitorSource(tableLoader(), HIGH_RATE, Long.MAX_VALUE), WatermarkStrategy.noWatermarks(), "TableChangeSource") .forceNonParallel(); @@ -176,8 +158,9 @@ void testSource() throws Exception { /** Check that the {@link MonitorSource} operator state is restored correctly. */ @Test void testStateRestore(@TempDir File savepointDir) throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + Table table = createTable(); + insert(table, 1, "a"); + TableLoader tableLoader = tableLoader(); Configuration config = new Configuration(); config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); @@ -185,8 +168,6 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); env.enableCheckpointing(1000); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); DataStream events = env.fromSource( new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), @@ -268,14 +249,12 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { @Test void testNotOneParallelismThrows() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + createTable(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); env.fromSource( - new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + new MonitorSource(tableLoader(), HIGH_RATE, Long.MAX_VALUE), WatermarkStrategy.noWatermarks(), "TableChangeSource") .setParallelism(2) @@ -289,14 +268,13 @@ void testNotOneParallelismThrows() { } @Test - void testMaxReadBack() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + void testMaxReadBack() throws IOException { + Table table = createTable(); + insert(table, 1, "a"); + insert(table, 2, "b"); + insert(table, 3, "c"); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); + TableLoader tableLoader = tableLoader(); MonitorSource.TableChangeIterator iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 1); @@ -316,12 +294,11 @@ void testMaxReadBack() { } @Test - void testSkipReplace() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + void testSkipReplace() throws IOException { + Table table = createTable(); + insert(table, 1, "a"); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); + TableLoader tableLoader = tableLoader(); MonitorSource.TableChangeIterator iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); @@ -330,7 +307,6 @@ void testSkipReplace() { assertThat(iterator.next().commitCount()).isEqualTo(1); // Create a DataOperations.REPLACE snapshot - Table table = tableLoader.loadTable(); DataFile dataFile = table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java index fba4a12d9c6b..a70d27279460 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -18,19 +18,13 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME_2; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; -import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.GROUP_VALUE_DEFAULT; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; import static org.assertj.core.api.Assertions.assertThat; -import java.io.IOException; import java.time.Duration; -import java.util.Iterator; import java.util.List; import java.util.stream.Stream; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -41,13 +35,14 @@ import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -56,34 +51,24 @@ class TestTriggerManager extends OperatorTestBase { private static final long DELAY = 10L; - private static final String NAME_1 = "name1"; - private static final String NAME_2 = "name2"; + private static final String[] TASKS = new String[] {"task0", "task1"}; private long processingTime = 0L; - private TriggerLockFactory lockFactory; private TriggerLockFactory.Lock lock; private TriggerLockFactory.Lock recoveringLock; + private String tableName; @BeforeEach void before() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - this.lockFactory = lockFactory(); - lockFactory.open(); - this.lock = lockFactory.createLock(); - this.recoveringLock = lockFactory.createRecoveryLock(); - lock.unlock(); - recoveringLock.unlock(); - MetricsReporterFactoryForTests.reset(); - } - - @AfterEach - void after() throws IOException { - lockFactory.close(); + Table table = createTable(); + this.lock = LOCK_FACTORY.createLock(); + this.recoveringLock = LOCK_FACTORY.createRecoveryLock(); + this.tableName = table.name(); } @Test void testCommitCount() throws Exception { TriggerManager manager = - manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().commitCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -104,8 +89,7 @@ void testCommitCount() throws Exception { @Test void testDataFileCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().dataFileCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().dataFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -126,9 +110,7 @@ void testDataFileCount() throws Exception { @Test void testDataFileSizeInBytes() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -147,9 +129,7 @@ void testDataFileSizeInBytes() throws Exception { @Test void testPosDeleteFileCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -170,9 +150,7 @@ void testPosDeleteFileCount() throws Exception { @Test void testPosDeleteRecordCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -196,9 +174,7 @@ void testPosDeleteRecordCount() throws Exception { @Test void testEqDeleteFileCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -219,9 +195,7 @@ void testEqDeleteFileCount() throws Exception { @Test void testEqDeleteRecordCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -241,8 +215,7 @@ void testEqDeleteRecordCount() throws Exception { void testTimeout() throws Exception { TriggerManager manager = manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); + tableLoader(), new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -281,7 +254,7 @@ void testTimeout() throws Exception { @Test void testStateRestore() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader); OperatorSubtaskState state; try (KeyedOneInputStreamOperatorTestHarness testHarness = @@ -319,7 +292,7 @@ void testStateRestore() throws Exception { @Test void testMinFireDelay() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader, DELAY, 1); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { @@ -339,7 +312,7 @@ void testMinFireDelay() throws Exception { @Test void testLockCheckDelay() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader, 1, DELAY); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { @@ -372,7 +345,7 @@ void testLockCheckDelay() throws Exception { @ParameterizedTest @MethodSource("parametersForTestRecovery") void testRecovery(boolean locked, boolean runningTask) throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader); OperatorSubtaskState state; try (KeyedOneInputStreamOperatorTestHarness testHarness = @@ -423,18 +396,14 @@ void testRecovery(boolean locked, boolean runningTask) throws Exception { ++processingTime; testHarness.setProcessingTime(processingTime); // Releasing lock will create a new snapshot, and we receive this in the trigger - expected.add( - Trigger.create( - processingTime, - (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()), - 0)); + expected.add(Trigger.create(processingTime, 0)); assertTriggers(testHarness.extractOutputValues(), expected); } } @Test void testTriggerMetrics() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); ManualSource source = @@ -444,8 +413,8 @@ void testTriggerMetrics() throws Exception { TriggerManager manager = new TriggerManager( tableLoader, - lockFactory, - Lists.newArrayList(NAME_1, NAME_2), + LOCK_FACTORY, + Lists.newArrayList(TASKS), Lists.newArrayList( new TriggerEvaluator.Builder().commitCount(2).build(), new TriggerEvaluator.Builder().commitCount(4).build()), @@ -455,7 +424,7 @@ void testTriggerMetrics() throws Exception { .dataStream() .keyBy(unused -> true) .process(manager) - .name(DUMMY_NAME) + .name(DUMMY_TASK_NAME) .forceNonParallel() .sinkTo(sink); @@ -471,7 +440,7 @@ void testTriggerMetrics() throws Exception { () -> { Long notingCounter = MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER); + ImmutableList.of(DUMMY_TASK_NAME, tableName, NOTHING_TO_TRIGGER)); return notingCounter != null && notingCounter.equals(1L); }); @@ -480,7 +449,8 @@ void testTriggerMetrics() throws Exception { // Wait until we receive the trigger assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter( + ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED))) .isEqualTo(1L); lock.unlock(); @@ -492,20 +462,22 @@ void testTriggerMetrics() throws Exception { assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); lock.unlock(); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter( + ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED))) .isEqualTo(2L); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter( + ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[1], "1", TRIGGERED))) .isEqualTo(1L); // Final check all the counters MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, -1L) - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, -1L) - .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 2L) - .put(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED, 1L) - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 1L) + new ImmutableMap.Builder, Long>() + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, RATE_LIMITER_TRIGGERED), -1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, CONCURRENT_RUN_THROTTLED), -1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED), 2L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[1], "1", TRIGGERED), 1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, NOTHING_TO_TRIGGER), 1L) .build()); } finally { closeJobClient(jobClient); @@ -514,7 +486,7 @@ void testTriggerMetrics() throws Exception { @Test void testRateLimiterMetrics() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); ManualSource source = @@ -527,7 +499,7 @@ void testRateLimiterMetrics() throws Exception { .dataStream() .keyBy(unused -> true) .process(manager) - .name(DUMMY_NAME) + .name(DUMMY_TASK_NAME) .forceNonParallel() .sinkTo(sink); @@ -548,7 +520,7 @@ void testRateLimiterMetrics() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED) + ImmutableList.of(DUMMY_TASK_NAME, tableName, RATE_LIMITER_TRIGGERED)) .equals(1L)); // Final check all the counters @@ -560,7 +532,7 @@ void testRateLimiterMetrics() throws Exception { @Test void testConcurrentRunMetrics() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); ManualSource source = @@ -573,7 +545,7 @@ void testConcurrentRunMetrics() throws Exception { .dataStream() .keyBy(unused -> true) .process(manager) - .name(DUMMY_NAME) + .name(DUMMY_TASK_NAME) .forceNonParallel() .sinkTo(sink); @@ -591,7 +563,7 @@ void testConcurrentRunMetrics() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED) + ImmutableList.of(DUMMY_TASK_NAME, tableName, CONCURRENT_RUN_THROTTLED)) .equals(1L)); // Final check all the counters @@ -611,15 +583,15 @@ private static Stream parametersForTestRecovery() { private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) { MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() + new ImmutableMap.Builder, Long>() .put( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, + ImmutableList.of(DUMMY_TASK_NAME, tableName, RATE_LIMITER_TRIGGERED), rateLimiterTrigger) .put( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, + ImmutableList.of(DUMMY_TASK_NAME, tableName, CONCURRENT_RUN_THROTTLED), concurrentRunTrigger) - .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 1L) - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED), 1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, NOTHING_TO_TRIGGER), 0L) .build()); } @@ -644,15 +616,20 @@ private void addEventAndCheckResult( private TriggerManager manager(TableLoader tableLoader, TriggerEvaluator evaluator) { return new TriggerManager( - tableLoader, lockFactory, Lists.newArrayList(NAME_1), Lists.newArrayList(evaluator), 1, 1); + tableLoader, + LOCK_FACTORY, + Lists.newArrayList(TASKS[0]), + Lists.newArrayList(evaluator), + 1, + 1); } private TriggerManager manager( TableLoader tableLoader, long minFireDelayMs, long lockCheckDelayMs) { return new TriggerManager( tableLoader, - lockFactory, - Lists.newArrayList(NAME_1), + LOCK_FACTORY, + Lists.newArrayList(TASKS[0]), Lists.newArrayList(new TriggerEvaluator.Builder().commitCount(2).build()), minFireDelayMs, lockCheckDelayMs); @@ -670,17 +647,6 @@ private static void assertTriggers(List expected, List actual) assertThat(actualTrigger.timestamp()).isEqualTo(expectedTrigger.timestamp()); assertThat(actualTrigger.taskId()).isEqualTo(expectedTrigger.taskId()); assertThat(actualTrigger.isRecovery()).isEqualTo(expectedTrigger.isRecovery()); - if (expectedTrigger.table() == null) { - assertThat(actualTrigger.table()).isNull(); - } else { - Iterator expectedSnapshots = expectedTrigger.table().snapshots().iterator(); - Iterator actualSnapshots = actualTrigger.table().snapshots().iterator(); - while (expectedSnapshots.hasNext()) { - assertThat(actualSnapshots.hasNext()).isTrue(); - assertThat(expectedSnapshots.next().snapshotId()) - .isEqualTo(actualSnapshots.next().snapshotId()); - } - } } } } From ec0eef45ebe11786072a71ae315c0637c2154862 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 15:13:21 +0100 Subject: [PATCH 092/313] Build: Bump mkdocs-material from 9.5.42 to 9.5.43 (#11455) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.42 to 9.5.43. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.5.42...9.5.43) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 5ace10851aa5..c5cc9261dd78 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.3.7 -mkdocs-material==9.5.42 +mkdocs-material==9.5.43 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.1 From 8357f65d23b8d3cc87f89b1aa818fcf4b5eb6d5d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 15:13:37 +0100 Subject: [PATCH 093/313] Build: Bump net.snowflake:snowflake-jdbc from 3.19.1 to 3.20.0 (#11447) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.19.1 to 3.20.0. - [Release notes](https://github.com/snowflakedb/snowflake-jdbc/releases) - [Changelog](https://github.com/snowflakedb/snowflake-jdbc/blob/master/CHANGELOG.rst) - [Commits](https://github.com/snowflakedb/snowflake-jdbc/compare/v3.19.1...v3.20.0) --- updated-dependencies: - dependency-name: net.snowflake:snowflake-jdbc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index b8c69377958e..d86379f0d0c3 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -80,7 +80,7 @@ pig = "0.17.0" roaringbitmap = "1.3.0" scala-collection-compat = "2.12.0" slf4j = "2.0.16" -snowflake-jdbc = "3.19.1" +snowflake-jdbc = "3.20.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.4" spark-hive35 = "3.5.2" From 329846875dcb9c58d01fee62ab778f91388bf45a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 15:15:02 +0100 Subject: [PATCH 094/313] Build: Bump kafka from 3.8.0 to 3.8.1 (#11449) Bumps `kafka` from 3.8.0 to 3.8.1. Updates `org.apache.kafka:kafka-clients` from 3.8.0 to 3.8.1 Updates `org.apache.kafka:connect-api` from 3.8.0 to 3.8.1 Updates `org.apache.kafka:connect-json` from 3.8.0 to 3.8.1 --- updated-dependencies: - dependency-name: org.apache.kafka:kafka-clients dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.kafka:connect-api dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.kafka:connect-json dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index d86379f0d0c3..a659b51cfe60 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -65,7 +65,7 @@ jaxb-runtime = "2.3.9" jetty = "11.0.24" junit = "5.11.3" junit-platform = "1.11.3" -kafka = "3.8.0" +kafka = "3.8.1" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" mockito = "4.11.0" From 9dcf0d3d32355e1e254d623ffaf093783eac5663 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 15:15:20 +0100 Subject: [PATCH 095/313] Build: Bump jackson-bom from 2.18.0 to 2.18.1 (#11448) Bumps `jackson-bom` from 2.18.0 to 2.18.1. Updates `com.fasterxml.jackson:jackson-bom` from 2.18.0 to 2.18.1 - [Commits](https://github.com/FasterXML/jackson-bom/compare/jackson-bom-2.18.0...jackson-bom-2.18.1) Updates `com.fasterxml.jackson.core:jackson-core` from 2.18.0 to 2.18.1 - [Commits](https://github.com/FasterXML/jackson-core/compare/jackson-core-2.18.0...jackson-core-2.18.1) Updates `com.fasterxml.jackson.core:jackson-databind` from 2.18.0 to 2.18.1 - [Commits](https://github.com/FasterXML/jackson/commits) Updates `com.fasterxml.jackson.core:jackson-annotations` from 2.18.0 to 2.18.1 - [Commits](https://github.com/FasterXML/jackson/commits) --- updated-dependencies: - dependency-name: com.fasterxml.jackson:jackson-bom dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: com.fasterxml.jackson.core:jackson-core dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: com.fasterxml.jackson.core:jackson-databind dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: com.fasterxml.jackson.core:jackson-annotations dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index a659b51cfe60..1e2ca79e94cf 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -52,7 +52,7 @@ httpcomponents-httpclient5 = "5.4.1" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" immutables-value = "2.10.1" -jackson-bom = "2.18.0" +jackson-bom = "2.18.1" jackson211 = { strictly = "2.11.4"} # see rich version usage explanation above jackson212 = { strictly = "2.12.3"} jackson213 = { strictly = "2.13.4"} From af5be32fbef36690d32b5e53c4153b709d8db188 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 4 Nov 2024 15:40:21 +0100 Subject: [PATCH 096/313] Core: Fix generated position delete file spec (#11458) --- core/src/test/java/org/apache/iceberg/FileGenerationUtil.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java index f66496ae6624..715a9d6db288 100644 --- a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -102,13 +102,13 @@ public static DeleteFile generateEqualityDeleteFile(Table table, StructLike part } public static DeleteFile generatePositionDeleteFile(Table table, DataFile dataFile) { - PartitionSpec spec = table.spec(); + PartitionSpec spec = table.specs().get(dataFile.specId()); StructLike partition = dataFile.partition(); LocationProvider locations = table.locationProvider(); String path = locations.newDataLocation(spec, partition, generateFileName()); long fileSize = generateFileSize(); Metrics metrics = generatePositionDeleteMetrics(dataFile); - return FileMetadata.deleteFileBuilder(table.spec()) + return FileMetadata.deleteFileBuilder(spec) .ofPositionDeletes() .withPath(path) .withPartition(partition) From ec269ee3ec0de4184eb536a6ef4f3523dc91332a Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 4 Nov 2024 19:35:08 +0100 Subject: [PATCH 097/313] API, Core: Add content offset and size to DeleteFile (#11446) --- .../org/apache/iceberg/AddedRowsScanTask.java | 3 +- .../java/org/apache/iceberg/DataFile.java | 15 ++++- .../java/org/apache/iceberg/DeleteFile.java | 22 ++++++++ .../iceberg/DeletedDataFileScanTask.java | 3 +- .../apache/iceberg/DeletedRowsScanTask.java | 7 ++- .../java/org/apache/iceberg/FileFormat.java | 1 + .../java/org/apache/iceberg/FileScanTask.java | 3 +- .../apache/iceberg/util/DeleteFileSet.java | 7 ++- .../org/apache/iceberg/util/ScanTaskUtil.java | 50 +++++++++++++++++ .../apache/iceberg/util/TestScanTaskUtil.java | 56 +++++++++++++++++++ .../java/org/apache/iceberg/BaseFile.java | 32 ++++++++++- .../org/apache/iceberg/BaseFileScanTask.java | 9 +-- .../java/org/apache/iceberg/BaseScan.java | 2 + .../org/apache/iceberg/ContentFileParser.java | 16 +++++- .../java/org/apache/iceberg/FileMetadata.java | 25 ++++++++- .../org/apache/iceberg/GenericDataFile.java | 4 +- .../org/apache/iceberg/GenericDeleteFile.java | 8 ++- .../java/org/apache/iceberg/ScanSummary.java | 3 +- .../org/apache/iceberg/SnapshotProducer.java | 10 ++++ .../org/apache/iceberg/SnapshotSummary.java | 5 +- .../java/org/apache/iceberg/V3Metadata.java | 16 +++++- .../iceberg/metrics/ScanMetricsUtil.java | 3 +- .../apache/iceberg/util/TableScanUtil.java | 4 +- .../apache/iceberg/FileGenerationUtil.java | 26 +++++++++ .../java/org/apache/iceberg/TestBase.java | 4 ++ .../apache/iceberg/TestContentFileParser.java | 35 +++++++++++- .../iceberg/TestManifestEncryption.java | 2 + .../apache/iceberg/TestManifestReader.java | 28 +++++++++- .../iceberg/TestManifestWriterVersions.java | 2 + .../iceberg/util/TestTableScanUtil.java | 16 ++++++ .../iceberg/flink/sink/CommitSummary.java | 4 +- .../sink/IcebergStreamWriterMetrics.java | 3 +- .../iceberg/flink/sink/CommitSummary.java | 4 +- .../sink/IcebergStreamWriterMetrics.java | 3 +- .../iceberg/flink/sink/CommitSummary.java | 4 +- .../sink/IcebergStreamWriterMetrics.java | 3 +- 36 files changed, 398 insertions(+), 40 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/util/ScanTaskUtil.java create mode 100644 api/src/test/java/org/apache/iceberg/util/TestScanTaskUtil.java diff --git a/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java b/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java index d48b268287c3..506e344d3660 100644 --- a/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java +++ b/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.util.List; +import org.apache.iceberg.util.ScanTaskUtil; /** * A scan task for inserts generated by adding a data file to the table. @@ -55,7 +56,7 @@ default ChangelogOperation operation() { @Override default long sizeBytes() { - return length() + deletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + return length() + ScanTaskUtil.contentSizeInBytes(deletes()); } @Override diff --git a/api/src/main/java/org/apache/iceberg/DataFile.java b/api/src/main/java/org/apache/iceberg/DataFile.java index 3c6d77f34d8f..ea6262afac85 100644 --- a/api/src/main/java/org/apache/iceberg/DataFile.java +++ b/api/src/main/java/org/apache/iceberg/DataFile.java @@ -104,12 +104,21 @@ public interface DataFile extends ContentFile { "referenced_data_file", StringType.get(), "Fully qualified location (URI with FS scheme) of a data file that all deletes reference"); + Types.NestedField CONTENT_OFFSET = + optional( + 144, "content_offset", LongType.get(), "The offset in the file where the content starts"); + Types.NestedField CONTENT_SIZE = + optional( + 145, + "content_size_in_bytes", + LongType.get(), + "The length of referenced content stored in the file"); int PARTITION_ID = 102; String PARTITION_NAME = "partition"; String PARTITION_DOC = "Partition data tuple, schema based on the partition spec"; - // NEXT ID TO ASSIGN: 144 + // NEXT ID TO ASSIGN: 146 static StructType getType(StructType partitionType) { // IDs start at 100 to leave room for changes to ManifestEntry @@ -131,7 +140,9 @@ static StructType getType(StructType partitionType) { SPLIT_OFFSETS, EQUALITY_IDS, SORT_ORDER_ID, - REFERENCED_DATA_FILE); + REFERENCED_DATA_FILE, + CONTENT_OFFSET, + CONTENT_SIZE); } /** diff --git a/api/src/main/java/org/apache/iceberg/DeleteFile.java b/api/src/main/java/org/apache/iceberg/DeleteFile.java index 8e17e60fcccf..340a00e36b17 100644 --- a/api/src/main/java/org/apache/iceberg/DeleteFile.java +++ b/api/src/main/java/org/apache/iceberg/DeleteFile.java @@ -42,4 +42,26 @@ default List splitOffsets() { default String referencedDataFile() { return null; } + + /** + * Returns the offset in the file where the content starts. + * + *

    The content offset is required for deletion vectors and points to the start of the deletion + * vector blob in the Puffin file, enabling direct access. This method always returns null for + * equality and position delete files. + */ + default Long contentOffset() { + return null; + } + + /** + * Returns the length of referenced content stored in the file. + * + *

    The content size is required for deletion vectors and indicates the size of the deletion + * vector blob in the Puffin file, enabling direct access. This method always returns null for + * equality and position delete files. + */ + default Long contentSizeInBytes() { + return null; + } } diff --git a/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java b/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java index 9edd6afd0cea..4b9c1704b9d2 100644 --- a/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java +++ b/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.util.List; +import org.apache.iceberg.util.ScanTaskUtil; /** * A scan task for deletes generated by removing a data file from the table. @@ -54,7 +55,7 @@ default ChangelogOperation operation() { @Override default long sizeBytes() { - return length() + existingDeletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + return length() + ScanTaskUtil.contentSizeInBytes(existingDeletes()); } @Override diff --git a/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java b/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java index 131edfddd349..1e0a52a53241 100644 --- a/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java +++ b/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.util.List; +import org.apache.iceberg.util.ScanTaskUtil; /** * A scan task for deletes generated by adding delete files to the table. @@ -63,9 +64,9 @@ default ChangelogOperation operation() { @Override default long sizeBytes() { - return length() - + addedDeletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum() - + existingDeletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long addedDeletesSize = ScanTaskUtil.contentSizeInBytes(addedDeletes()); + long existingDeletesSize = ScanTaskUtil.contentSizeInBytes(existingDeletes()); + return length() + addedDeletesSize + existingDeletesSize; } @Override diff --git a/api/src/main/java/org/apache/iceberg/FileFormat.java b/api/src/main/java/org/apache/iceberg/FileFormat.java index d662437d5ddb..6b41aec42c3e 100644 --- a/api/src/main/java/org/apache/iceberg/FileFormat.java +++ b/api/src/main/java/org/apache/iceberg/FileFormat.java @@ -24,6 +24,7 @@ /** Enum of supported file formats. */ public enum FileFormat { + PUFFIN("puffin", false), ORC("orc", true), PARQUET("parquet", true), AVRO("avro", true), diff --git a/api/src/main/java/org/apache/iceberg/FileScanTask.java b/api/src/main/java/org/apache/iceberg/FileScanTask.java index 5fb4b55459e3..94f153e56052 100644 --- a/api/src/main/java/org/apache/iceberg/FileScanTask.java +++ b/api/src/main/java/org/apache/iceberg/FileScanTask.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import java.util.List; +import org.apache.iceberg.util.ScanTaskUtil; /** A scan task over a range of bytes in a single data file. */ public interface FileScanTask extends ContentScanTask, SplittableScanTask { @@ -36,7 +37,7 @@ default Schema schema() { @Override default long sizeBytes() { - return length() + deletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + return length() + ScanTaskUtil.contentSizeInBytes(deletes()); } @Override diff --git a/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java b/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java index bbe9824963fc..06ddd1869ace 100644 --- a/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java +++ b/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java @@ -97,13 +97,14 @@ public boolean equals(Object o) { } DeleteFileWrapper that = (DeleteFileWrapper) o; - // this needs to be updated once deletion vector support is added - return Objects.equals(file.location(), that.file.location()); + return Objects.equals(file.location(), that.file.location()) + && Objects.equals(file.contentOffset(), that.file.contentOffset()) + && Objects.equals(file.contentSizeInBytes(), that.file.contentSizeInBytes()); } @Override public int hashCode() { - return Objects.hashCode(file.location()); + return Objects.hash(file.location(), file.contentOffset(), file.contentSizeInBytes()); } @Override diff --git a/api/src/main/java/org/apache/iceberg/util/ScanTaskUtil.java b/api/src/main/java/org/apache/iceberg/util/ScanTaskUtil.java new file mode 100644 index 000000000000..276aae6e2caf --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/util/ScanTaskUtil.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; + +public class ScanTaskUtil { + + private ScanTaskUtil() {} + + public static long contentSizeInBytes(ContentFile file) { + if (file.content() == FileContent.DATA) { + return file.fileSizeInBytes(); + } else { + DeleteFile deleteFile = (DeleteFile) file; + return isDV(deleteFile) ? deleteFile.contentSizeInBytes() : deleteFile.fileSizeInBytes(); + } + } + + public static long contentSizeInBytes(Iterable> files) { + long size = 0L; + for (ContentFile file : files) { + size += contentSizeInBytes(file); + } + return size; + } + + private static boolean isDV(DeleteFile deleteFile) { + return deleteFile.format() == FileFormat.PUFFIN; + } +} diff --git a/api/src/test/java/org/apache/iceberg/util/TestScanTaskUtil.java b/api/src/test/java/org/apache/iceberg/util/TestScanTaskUtil.java new file mode 100644 index 000000000000..a449cf20a65b --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/util/TestScanTaskUtil.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +public class TestScanTaskUtil { + + @Test + public void testContentSize() { + DeleteFile dv1 = mockDV("dv1.puffin", 20L, 25L, "data1.parquet"); + DeleteFile dv2 = mockDV("dv2.puffin", 4L, 15L, "data2.parquet"); + + long size1 = ScanTaskUtil.contentSizeInBytes(ImmutableList.of()); + assertThat(size1).isEqualTo(0); + + long size2 = ScanTaskUtil.contentSizeInBytes(ImmutableList.of(dv1)); + assertThat(size2).isEqualTo(25L); + + long size3 = ScanTaskUtil.contentSizeInBytes(ImmutableList.of(dv1, dv2)); + assertThat(size3).isEqualTo(40L); + } + + private static DeleteFile mockDV( + String location, long contentOffset, long contentSize, String referencedDataFile) { + DeleteFile mockFile = Mockito.mock(DeleteFile.class); + Mockito.when(mockFile.format()).thenReturn(FileFormat.PUFFIN); + Mockito.when(mockFile.location()).thenReturn(location); + Mockito.when(mockFile.contentOffset()).thenReturn(contentOffset); + Mockito.when(mockFile.contentSizeInBytes()).thenReturn(contentSize); + Mockito.when(mockFile.referencedDataFile()).thenReturn(referencedDataFile); + return mockFile; + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseFile.java b/core/src/main/java/org/apache/iceberg/BaseFile.java index f4fd94724e95..e9724637dfa3 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFile.java +++ b/core/src/main/java/org/apache/iceberg/BaseFile.java @@ -81,6 +81,8 @@ public PartitionData copy() { private byte[] keyMetadata = null; private Integer sortOrderId; private String referencedDataFile = null; + private Long contentOffset = null; + private Long contentSizeInBytes = null; // cached schema private transient Schema avroSchema = null; @@ -110,6 +112,8 @@ public PartitionData copy() { DataFile.EQUALITY_IDS, DataFile.SORT_ORDER_ID, DataFile.REFERENCED_DATA_FILE, + DataFile.CONTENT_OFFSET, + DataFile.CONTENT_SIZE, MetadataColumns.ROW_POSITION); /** Used by Avro reflection to instantiate this class when reading manifest files. */ @@ -152,7 +156,9 @@ public PartitionData copy() { int[] equalityFieldIds, Integer sortOrderId, ByteBuffer keyMetadata, - String referencedDataFile) { + String referencedDataFile, + Long contentOffset, + Long contentSizeInBytes) { super(BASE_TYPE.fields().size()); this.partitionSpecId = specId; this.content = content; @@ -182,6 +188,8 @@ public PartitionData copy() { this.sortOrderId = sortOrderId; this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); this.referencedDataFile = referencedDataFile; + this.contentOffset = contentOffset; + this.contentSizeInBytes = contentSizeInBytes; } /** @@ -235,6 +243,8 @@ public PartitionData copy() { this.dataSequenceNumber = toCopy.dataSequenceNumber; this.fileSequenceNumber = toCopy.fileSequenceNumber; this.referencedDataFile = toCopy.referencedDataFile; + this.contentOffset = toCopy.contentOffset; + this.contentSizeInBytes = toCopy.contentSizeInBytes; } /** Constructor for Java serialization. */ @@ -347,6 +357,12 @@ protected void internalSet(int pos, T value) { this.referencedDataFile = value != null ? value.toString() : null; return; case 18: + this.contentOffset = (Long) value; + return; + case 19: + this.contentSizeInBytes = (Long) value; + return; + case 20: this.fileOrdinal = (long) value; return; default: @@ -398,6 +414,10 @@ private Object getByPos(int basePos) { case 17: return referencedDataFile; case 18: + return contentOffset; + case 19: + return contentSizeInBytes; + case 20: return fileOrdinal; default: throw new UnsupportedOperationException("Unknown field ordinal: " + basePos); @@ -528,6 +548,14 @@ public String referencedDataFile() { return referencedDataFile; } + public Long contentOffset() { + return contentOffset; + } + + public Long contentSizeInBytes() { + return contentSizeInBytes; + } + private static Map copyMap(Map map, Set keys) { return keys == null ? SerializableMap.copyOf(map) : SerializableMap.filteredCopyOf(map, keys); } @@ -580,6 +608,8 @@ public String toString() { .add("data_sequence_number", dataSequenceNumber == null ? "null" : dataSequenceNumber) .add("file_sequence_number", fileSequenceNumber == null ? "null" : fileSequenceNumber) .add("referenced_data_file", referencedDataFile == null ? "null" : referencedDataFile) + .add("content_offset", contentOffset == null ? "null" : contentOffset) + .add("content_size_in_bytes", contentSizeInBytes == null ? "null" : contentSizeInBytes) .toString(); } } diff --git a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java index 2469395021d4..aa37f40be7c0 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java @@ -23,6 +23,7 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.ScanTaskUtil; public class BaseFileScanTask extends BaseContentScanTask implements FileScanTask { @@ -79,7 +80,7 @@ private long deletesSizeBytes() { if (deletesSizeBytes == 0L && deletes.length > 0) { long size = 0L; for (DeleteFile deleteFile : deletes) { - size += deleteFile.fileSizeInBytes(); + size += ScanTaskUtil.contentSizeInBytes(deleteFile); } this.deletesSizeBytes = size; } @@ -180,11 +181,7 @@ public SplitScanTask merge(ScanTask other) { private long deletesSizeBytes() { if (deletesSizeBytes == 0L && fileScanTask.filesCount() > 1) { - long size = 0L; - for (DeleteFile deleteFile : fileScanTask.deletes()) { - size += deleteFile.fileSizeInBytes(); - } - this.deletesSizeBytes = size; + this.deletesSizeBytes = ScanTaskUtil.contentSizeInBytes(fileScanTask.deletes()); } return deletesSizeBytes; diff --git a/core/src/main/java/org/apache/iceberg/BaseScan.java b/core/src/main/java/org/apache/iceberg/BaseScan.java index a011d03d59ad..618b2e95f29f 100644 --- a/core/src/main/java/org/apache/iceberg/BaseScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseScan.java @@ -78,6 +78,8 @@ abstract class BaseScan> "key_metadata", "split_offsets", "referenced_data_file", + "content_offset", + "content_size_in_bytes", "equality_ids"); protected static final List DELETE_SCAN_WITH_STATS_COLUMNS = diff --git a/core/src/main/java/org/apache/iceberg/ContentFileParser.java b/core/src/main/java/org/apache/iceberg/ContentFileParser.java index 96dfa5586c31..e6d7c8043f3f 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileParser.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileParser.java @@ -46,6 +46,8 @@ class ContentFileParser { private static final String EQUALITY_IDS = "equality-ids"; private static final String SORT_ORDER_ID = "sort-order-id"; private static final String REFERENCED_DATA_FILE = "referenced-data-file"; + private static final String CONTENT_OFFSET = "content-offset"; + private static final String CONTENT_SIZE = "content-size-in-bytes"; private ContentFileParser() {} @@ -116,6 +118,14 @@ static void toJson(ContentFile contentFile, PartitionSpec spec, JsonGenerator if (deleteFile.referencedDataFile() != null) { generator.writeStringField(REFERENCED_DATA_FILE, deleteFile.referencedDataFile()); } + + if (deleteFile.contentOffset() != null) { + generator.writeNumberField(CONTENT_OFFSET, deleteFile.contentOffset()); + } + + if (deleteFile.contentSizeInBytes() != null) { + generator.writeNumberField(CONTENT_SIZE, deleteFile.contentSizeInBytes()); + } } generator.writeEndObject(); @@ -155,6 +165,8 @@ static ContentFile fromJson(JsonNode jsonNode, PartitionSpec spec) { int[] equalityFieldIds = JsonUtil.getIntArrayOrNull(EQUALITY_IDS, jsonNode); Integer sortOrderId = JsonUtil.getIntOrNull(SORT_ORDER_ID, jsonNode); String referencedDataFile = JsonUtil.getStringOrNull(REFERENCED_DATA_FILE, jsonNode); + Long contentOffset = JsonUtil.getLongOrNull(CONTENT_OFFSET, jsonNode); + Long contentSizeInBytes = JsonUtil.getLongOrNull(CONTENT_SIZE, jsonNode); if (fileContent == FileContent.DATA) { return new GenericDataFile( @@ -180,7 +192,9 @@ static ContentFile fromJson(JsonNode jsonNode, PartitionSpec spec) { sortOrderId, splitOffsets, keyMetadata, - referencedDataFile); + referencedDataFile, + contentOffset, + contentSizeInBytes); } } diff --git a/core/src/main/java/org/apache/iceberg/FileMetadata.java b/core/src/main/java/org/apache/iceberg/FileMetadata.java index ef229593bcab..7bb8d886dd16 100644 --- a/core/src/main/java/org/apache/iceberg/FileMetadata.java +++ b/core/src/main/java/org/apache/iceberg/FileMetadata.java @@ -60,6 +60,8 @@ public static class Builder { private Integer sortOrderId = null; private List splitOffsets = null; private String referencedDataFile = null; + private Long contentOffset = null; + private Long contentSizeInBytes = null; Builder(PartitionSpec spec) { this.spec = spec; @@ -230,6 +232,16 @@ public Builder withReferencedDataFile(CharSequence newReferencedDataFile) { return this; } + public Builder withContentOffset(long newContentOffset) { + this.contentOffset = newContentOffset; + return this; + } + + public Builder withContentSizeInBytes(long newContentSizeInBytes) { + this.contentSizeInBytes = newContentSizeInBytes; + return this; + } + public DeleteFile build() { Preconditions.checkArgument(filePath != null, "File path is required"); if (format == null) { @@ -240,6 +252,15 @@ public DeleteFile build() { Preconditions.checkArgument(fileSizeInBytes >= 0, "File size is required"); Preconditions.checkArgument(recordCount >= 0, "Record count is required"); + if (format == FileFormat.PUFFIN) { + Preconditions.checkArgument(contentOffset != null, "Content offset is required for DV"); + Preconditions.checkArgument(contentSizeInBytes != null, "Content size is required for DV"); + } else { + Preconditions.checkArgument(contentOffset == null, "Content offset can only be set for DV"); + Preconditions.checkArgument( + contentSizeInBytes == null, "Content size can only be set for DV"); + } + switch (content) { case POSITION_DELETES: Preconditions.checkArgument( @@ -273,7 +294,9 @@ public DeleteFile build() { sortOrderId, splitOffsets, keyMetadata, - referencedDataFile); + referencedDataFile, + contentOffset, + contentSizeInBytes); } } } diff --git a/core/src/main/java/org/apache/iceberg/GenericDataFile.java b/core/src/main/java/org/apache/iceberg/GenericDataFile.java index aa34cd22cdaa..a61cc1e0fb72 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDataFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java @@ -65,7 +65,9 @@ class GenericDataFile extends BaseFile implements DataFile { null /* no equality field IDs */, sortOrderId, keyMetadata, - null /* no referenced data file */); + null /* no referenced data file */, + null /* no content offset */, + null /* no content size */); } /** diff --git a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java index 05eb7c97dbab..9205551f24b3 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java @@ -49,7 +49,9 @@ class GenericDeleteFile extends BaseFile implements DeleteFile { Integer sortOrderId, List splitOffsets, ByteBuffer keyMetadata, - String referencedDataFile) { + String referencedDataFile, + Long contentOffset, + Long contentSizeInBytes) { super( specId, content, @@ -68,7 +70,9 @@ class GenericDeleteFile extends BaseFile implements DeleteFile { equalityFieldIds, sortOrderId, keyMetadata, - referencedDataFile); + referencedDataFile, + contentOffset, + contentSizeInBytes); } /** diff --git a/core/src/main/java/org/apache/iceberg/ScanSummary.java b/core/src/main/java/org/apache/iceberg/ScanSummary.java index 1ea171c5b2c3..5f8e66c0b450 100644 --- a/core/src/main/java/org/apache/iceberg/ScanSummary.java +++ b/core/src/main/java/org/apache/iceberg/ScanSummary.java @@ -47,6 +47,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.ScanTaskUtil; public class ScanSummary { private ScanSummary() {} @@ -294,7 +295,7 @@ PartitionMetrics updateFromCounts( private PartitionMetrics updateFromFile(ContentFile file, Long timestampMillis) { this.fileCount += 1; this.recordCount += file.recordCount(); - this.totalSize += file.fileSizeInBytes(); + this.totalSize += ScanTaskUtil.contentSizeInBytes(file); if (timestampMillis != null && (dataTimestampMillis == null || dataTimestampMillis < timestampMillis)) { this.dataTimestampMillis = timestampMillis; diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index daf1c3d72b89..45b71d654344 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -928,5 +928,15 @@ public Integer sortOrderId() { public String referencedDataFile() { return deleteFile.referencedDataFile(); } + + @Override + public Long contentOffset() { + return deleteFile.contentOffset(); + } + + @Override + public Long contentSizeInBytes() { + return deleteFile.contentSizeInBytes(); + } } } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java index 22c9df2a8eaf..ad832a5e78e2 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java @@ -25,6 +25,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; 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.ScanTaskUtil; public class SnapshotSummary { public static final String ADDED_FILES_PROP = "added-data-files"; @@ -275,7 +276,7 @@ void addTo(ImmutableMap.Builder builder) { } void addedFile(ContentFile file) { - this.addedSize += file.fileSizeInBytes(); + this.addedSize += ScanTaskUtil.contentSizeInBytes(file); switch (file.content()) { case DATA: this.addedFiles += 1; @@ -298,7 +299,7 @@ void addedFile(ContentFile file) { } void removedFile(ContentFile file) { - this.removedSize += file.fileSizeInBytes(); + this.removedSize += ScanTaskUtil.contentSizeInBytes(file); switch (file.content()) { case DATA: this.removedFiles += 1; diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java index a418a868564e..70461ac74a70 100644 --- a/core/src/main/java/org/apache/iceberg/V3Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -275,7 +275,9 @@ static Types.StructType fileType(Types.StructType partitionType) { DataFile.SPLIT_OFFSETS, DataFile.EQUALITY_IDS, DataFile.SORT_ORDER_ID, - DataFile.REFERENCED_DATA_FILE); + DataFile.REFERENCED_DATA_FILE, + DataFile.CONTENT_OFFSET, + DataFile.CONTENT_SIZE); } static class IndexedManifestEntry> @@ -455,6 +457,18 @@ public Object get(int pos) { } else { return null; } + case 17: + if (wrapped.content() == FileContent.POSITION_DELETES) { + return ((DeleteFile) wrapped).contentOffset(); + } else { + return null; + } + case 18: + if (wrapped.content() == FileContent.POSITION_DELETES) { + return ((DeleteFile) wrapped).contentSizeInBytes(); + } else { + return null; + } } throw new IllegalArgumentException("Unknown field ordinal: " + pos); } diff --git a/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsUtil.java b/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsUtil.java index c5aa6e1dd673..1ba891f58474 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsUtil.java +++ b/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsUtil.java @@ -21,6 +21,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileContent; +import org.apache.iceberg.util.ScanTaskUtil; public class ScanMetricsUtil { @@ -43,7 +44,7 @@ public static void fileTask(ScanMetrics metrics, DataFile dataFile, DeleteFile[] long deletesSizeInBytes = 0L; for (DeleteFile deleteFile : deleteFiles) { - deletesSizeInBytes += deleteFile.fileSizeInBytes(); + deletesSizeInBytes += ScanTaskUtil.contentSizeInBytes(deleteFile); } metrics.totalDeleteFileSizeInBytes().increment(deletesSizeInBytes); diff --git a/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java b/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java index e2dbcb61e9b7..2d80e88ae328 100644 --- a/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java @@ -25,7 +25,6 @@ import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.BaseScanTaskGroup; import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MergeableScanTask; @@ -92,8 +91,7 @@ public static CloseableIterable planTasks( Function weightFunc = file -> Math.max( - file.length() - + file.deletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(), + file.length() + ScanTaskUtil.contentSizeInBytes(file.deletes()), (1 + file.deletes().size()) * openFileCost); return CloseableIterable.transform( diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java index 715a9d6db288..b210cfcd4fa7 100644 --- a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -101,6 +101,24 @@ public static DeleteFile generateEqualityDeleteFile(Table table, StructLike part .build(); } + public static DeleteFile generateDV(Table table, DataFile dataFile) { + PartitionSpec spec = table.specs().get(dataFile.specId()); + long fileSize = generateFileSize(); + long cardinality = generateRowCount(); + long offset = generateContentOffset(); + long length = generateContentLength(); + return FileMetadata.deleteFileBuilder(spec) + .ofPositionDeletes() + .withPath("/path/to/delete-" + UUID.randomUUID() + ".puffin") + .withFileSizeInBytes(fileSize) + .withPartition(dataFile.partition()) + .withRecordCount(cardinality) + .withReferencedDataFile(dataFile.location()) + .withContentOffset(offset) + .withContentSizeInBytes(length) + .build(); + } + public static DeleteFile generatePositionDeleteFile(Table table, DataFile dataFile) { PartitionSpec spec = table.specs().get(dataFile.specId()); StructLike partition = dataFile.partition(); @@ -229,6 +247,14 @@ private static long generateFileSize() { return random().nextInt(50_000); } + private static long generateContentOffset() { + return random().nextInt(1_000_000); + } + + private static long generateContentLength() { + return random().nextInt(10_000); + } + private static Pair generateBounds(PrimitiveType type, MetricsMode mode) { Comparator cmp = Comparators.forType(type); Object value1 = generateBound(type, mode); diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 45441631900c..9813d02910a6 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -666,6 +666,10 @@ protected DeleteFile newDeleteFileWithRef(DataFile dataFile) { .build(); } + protected DeleteFile newDV(DataFile dataFile) { + return FileGenerationUtil.generateDV(table, dataFile); + } + protected DeleteFile newEqualityDeleteFile(int specId, String partitionPath, int... fieldIds) { PartitionSpec spec = table.specs().get(specId); return FileMetadata.deleteFileBuilder(spec) diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index fbe473931659..0c98e8448745 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -198,6 +198,7 @@ private static DataFile dataFileWithAllOptional(PartitionSpec spec) { private static Stream provideSpecAndDeleteFile() { return Stream.of( + Arguments.of(TestBase.SPEC, dv(TestBase.SPEC), dvJson()), Arguments.of( PartitionSpec.unpartitioned(), deleteFileWithRequiredOnly(PartitionSpec.unpartitioned()), @@ -233,7 +234,9 @@ private static DeleteFile deleteFileWithDataRef(PartitionSpec spec) { null, null, null, - "/path/to/data/file.parquet"); + "/path/to/data/file.parquet", + null, + null); } private static String deleteFileWithDataRefJson() { @@ -242,6 +245,32 @@ private static String deleteFileWithDataRefJson() { + "\"record-count\":10,\"referenced-data-file\":\"/path/to/data/file.parquet\"}"; } + private static DeleteFile dv(PartitionSpec spec) { + PartitionData partitionData = new PartitionData(spec.partitionType()); + partitionData.set(0, 4); + return new GenericDeleteFile( + spec.specId(), + FileContent.POSITION_DELETES, + "/path/to/delete.puffin", + FileFormat.PUFFIN, + partitionData, + 1234, + new Metrics(10L, null, null, null, null), + null, + null, + null, + null, + "/path/to/data/file.parquet", + 4L, + 40L); + } + + private static String dvJson() { + return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete.puffin\"," + + "\"file-format\":\"PUFFIN\",\"partition\":{\"1000\":4},\"file-size-in-bytes\":1234,\"record-count\":10," + + "\"referenced-data-file\":\"/path/to/data/file.parquet\",\"content-offset\":4,\"content-size-in-bytes\":40}"; + } + private static DeleteFile deleteFileWithRequiredOnly(PartitionSpec spec) { PartitionData partitionData = null; if (spec.isPartitioned()) { @@ -261,6 +290,8 @@ private static DeleteFile deleteFileWithRequiredOnly(PartitionSpec spec) { null, null, null, + null, + null, null); } @@ -301,6 +332,8 @@ private static DeleteFile deleteFileWithAllOptional(PartitionSpec spec) { 1, Collections.singletonList(128L), ByteBuffer.wrap(new byte[16]), + null, + null, null); } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java index 1f29c0e5b85c..01d38dc129c9 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java @@ -111,6 +111,8 @@ public class TestManifestEncryption { SORT_ORDER_ID, null, CONTENT_KEY_METADATA, + null, + null, null); private static final EncryptionManager ENCRYPTION_MANAGER = diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java index 4652da943003..63c6779298e0 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java @@ -130,7 +130,7 @@ public void testDataFilePositions() throws IOException { long expectedPos = 0L; for (DataFile file : reader) { assertThat(file.pos()).as("Position should match").isEqualTo(expectedPos); - assertThat(((BaseFile) file).get(18)) + assertThat(((BaseFile) file).get(20)) .as("Position from field index should match") .isEqualTo(expectedPos); expectedPos += 1; @@ -158,7 +158,7 @@ public void testDeleteFilePositions() throws IOException { long expectedPos = 0L; for (DeleteFile file : reader) { assertThat(file.pos()).as("Position should match").isEqualTo(expectedPos); - assertThat(((BaseFile) file).get(18)) + assertThat(((BaseFile) file).get(20)) .as("Position from field index should match") .isEqualTo(expectedPos); expectedPos += 1; @@ -199,6 +199,30 @@ public void testDeleteFilesWithReferences() throws IOException { } } + @TestTemplate + public void testDVs() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + DeleteFile dv1 = newDV(FILE_A); + DeleteFile dv2 = newDV(FILE_B); + ManifestFile manifest = writeDeleteManifest(formatVersion, 1000L, dv1, dv2); + try (ManifestReader reader = + ManifestFiles.readDeleteManifest(manifest, FILE_IO, table.specs())) { + for (DeleteFile dv : reader) { + if (dv.location().equals(dv1.location())) { + assertThat(dv.location()).isEqualTo(dv1.location()); + assertThat(dv.referencedDataFile()).isEqualTo(FILE_A.location()); + assertThat(dv.contentOffset()).isEqualTo(dv1.contentOffset()); + assertThat(dv.contentSizeInBytes()).isEqualTo(dv1.contentSizeInBytes()); + } else { + assertThat(dv.location()).isEqualTo(dv2.location()); + assertThat(dv.referencedDataFile()).isEqualTo(FILE_B.location()); + assertThat(dv.contentOffset()).isEqualTo(dv2.contentOffset()); + assertThat(dv.contentSizeInBytes()).isEqualTo(dv2.contentSizeInBytes()); + } + } + } + } + @TestTemplate public void testDataFileSplitOffsetsNullWhenInvalid() throws IOException { DataFile invalidOffset = diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 88dcc6ff9ca4..9abe7c426f32 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -97,6 +97,8 @@ public class TestManifestWriterVersions { SORT_ORDER_ID, null, null, + null, + null, null); @TempDir private Path temp; diff --git a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java index eb713a4d2e0b..8f8343733525 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java @@ -31,6 +31,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MergeableScanTask; import org.apache.iceberg.MockFileScanTask; @@ -74,6 +75,13 @@ private DataFile dataFileWithSize(long size) { return mockFile; } + private DeleteFile dvWithSize(long size) { + DeleteFile mockDeleteFile = Mockito.mock(DeleteFile.class); + Mockito.when(mockDeleteFile.format()).thenReturn(FileFormat.PUFFIN); + Mockito.when(mockDeleteFile.contentSizeInBytes()).thenReturn(size); + return mockDeleteFile; + } + private DeleteFile[] deleteFilesWithSizes(long... sizes) { return Arrays.stream(sizes) .mapToObj( @@ -85,6 +93,14 @@ private DeleteFile[] deleteFilesWithSizes(long... sizes) { .toArray(DeleteFile[]::new); } + @Test + public void testFileScanTaskSizeEstimation() { + DataFile dataFile = dataFileWithSize(100L); + DeleteFile dv = dvWithSize(20L); + MockFileScanTask task = new MockFileScanTask(dataFile, new DeleteFile[] {dv}); + assertThat(task.sizeBytes()).isEqualTo(120L); + } + @Test public void testPlanTaskWithDeleteFiles() { List testFiles = diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java index 9a2f57181708..2109c91bddf7 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.util.ScanTaskUtil; class CommitSummary { @@ -50,7 +51,8 @@ class CommitSummary { .forEach( deleteFile -> { deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); - deleteFilesByteCount.addAndGet(deleteFile.fileSizeInBytes()); + long deleteBytes = ScanTaskUtil.contentSizeInBytes(deleteFile); + deleteFilesByteCount.addAndGet(deleteBytes); }); }); } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index ce2a6c583fdf..ab458ad2e7cb 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -26,6 +26,7 @@ import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.MetricGroup; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.util.ScanTaskUtil; class IcebergStreamWriterMetrics { // 1,024 reservoir size should cost about 8KB, which is quite small. @@ -79,7 +80,7 @@ void updateFlushResult(WriteResult result) { Arrays.stream(result.deleteFiles()) .forEach( deleteFile -> { - deleteFilesSizeHistogram.update(deleteFile.fileSizeInBytes()); + deleteFilesSizeHistogram.update(ScanTaskUtil.contentSizeInBytes(deleteFile)); }); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java index 9a2f57181708..2109c91bddf7 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.util.ScanTaskUtil; class CommitSummary { @@ -50,7 +51,8 @@ class CommitSummary { .forEach( deleteFile -> { deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); - deleteFilesByteCount.addAndGet(deleteFile.fileSizeInBytes()); + long deleteBytes = ScanTaskUtil.contentSizeInBytes(deleteFile); + deleteFilesByteCount.addAndGet(deleteBytes); }); }); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index ce2a6c583fdf..ab458ad2e7cb 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -26,6 +26,7 @@ import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.MetricGroup; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.util.ScanTaskUtil; class IcebergStreamWriterMetrics { // 1,024 reservoir size should cost about 8KB, which is quite small. @@ -79,7 +80,7 @@ void updateFlushResult(WriteResult result) { Arrays.stream(result.deleteFiles()) .forEach( deleteFile -> { - deleteFilesSizeHistogram.update(deleteFile.fileSizeInBytes()); + deleteFilesSizeHistogram.update(ScanTaskUtil.contentSizeInBytes(deleteFile)); }); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java index 9a2f57181708..2109c91bddf7 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.util.ScanTaskUtil; class CommitSummary { @@ -50,7 +51,8 @@ class CommitSummary { .forEach( deleteFile -> { deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); - deleteFilesByteCount.addAndGet(deleteFile.fileSizeInBytes()); + long deleteBytes = ScanTaskUtil.contentSizeInBytes(deleteFile); + deleteFilesByteCount.addAndGet(deleteBytes); }); }); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java index ce2a6c583fdf..ab458ad2e7cb 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -26,6 +26,7 @@ import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.MetricGroup; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.util.ScanTaskUtil; class IcebergStreamWriterMetrics { // 1,024 reservoir size should cost about 8KB, which is quite small. @@ -79,7 +80,7 @@ void updateFlushResult(WriteResult result) { Arrays.stream(result.deleteFiles()) .forEach( deleteFile -> { - deleteFilesSizeHistogram.update(deleteFile.fileSizeInBytes()); + deleteFilesSizeHistogram.update(ScanTaskUtil.contentSizeInBytes(deleteFile)); }); } From 7cc16fa94d7cd4e19397e9b4fba62185e0fa5eac Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 4 Nov 2024 13:27:40 -0600 Subject: [PATCH 098/313] Revert "Build: Bump parquet from 1.13.1 to 1.14.3 (#11264)" (#11462) This reverts commit b8c2b20237bc9309d34dc96c473e9941d1b2ad58. apache/parquet-java#3040 Was discovered by @pan3793 in Parquet 1.14.(0,1,2,3). --- .../TestMetadataTableReadableMetrics.java | 22 +++++++++---------- .../TestMetadataTableReadableMetrics.java | 22 +++++++++---------- .../TestMetadataTableReadableMetrics.java | 22 +++++++++---------- gradle/libs.versions.toml | 2 +- 4 files changed, 34 insertions(+), 34 deletions(-) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 3aa2a7c0667f..40dfda723749 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -217,27 +217,27 @@ public void testPrimitiveColumns() throws Exception { Row binaryCol = Row.of( - 55L, + 52L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(36L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(91L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(91L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); Row fixedCol = Row.of( - 47L, + 44L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(77L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(77L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(85L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(85L, 4L, 0L, null, "1", "2"); + Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); List expected = Lists.newArrayList( @@ -289,8 +289,8 @@ public void testSelectNestedValues() throws Exception { public void testNestedValues() throws Exception { createNestedTable(); - Row leafDoubleCol = Row.of(50L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(57L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); TestHelpers.assertRows( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 9cdcb72c12cf..9cf953342a18 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -217,27 +217,27 @@ public void testPrimitiveColumns() throws Exception { Row binaryCol = Row.of( - 55L, + 52L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(36L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(91L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(91L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); Row fixedCol = Row.of( - 47L, + 44L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(77L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(77L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(85L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(85L, 4L, 0L, null, "1", "2"); + Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); List expected = Lists.newArrayList( @@ -289,8 +289,8 @@ public void testSelectNestedValues() throws Exception { public void testNestedValues() throws Exception { createNestedTable(); - Row leafDoubleCol = Row.of(50L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(57L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); TestHelpers.assertRows( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 9cdcb72c12cf..9cf953342a18 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -217,27 +217,27 @@ public void testPrimitiveColumns() throws Exception { Row binaryCol = Row.of( - 55L, + 52L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(36L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(91L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(91L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); + Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); + Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); Row fixedCol = Row.of( - 47L, + 44L, 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(77L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(77L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(85L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(85L, 4L, 0L, null, "1", "2"); + Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); + Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); + Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); + Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); List expected = Lists.newArrayList( @@ -289,8 +289,8 @@ public void testSelectNestedValues() throws Exception { public void testNestedValues() throws Exception { createNestedTable(); - Row leafDoubleCol = Row.of(50L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(57L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); TestHelpers.assertRows( diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 1e2ca79e94cf..091d0a9ec5f9 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -75,7 +75,7 @@ netty-buffer = "4.1.114.Final" netty-buffer-compat = "4.1.114.Final" object-client-bundle = "3.3.2" orc = "1.9.4" -parquet = "1.14.3" +parquet = "1.13.1" pig = "0.17.0" roaringbitmap = "1.3.0" scala-collection-compat = "2.12.0" From d0cca384a01172b5133bf7e207d94e374ed0c2ed Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 4 Nov 2024 21:22:50 +0100 Subject: [PATCH 099/313] Spark 3.5: Preserve data file reference during manifest rewrites (#11457) --- .../apache/iceberg/FileGenerationUtil.java | 17 +++++ .../iceberg/spark/SparkContentFile.java | 9 +++ .../actions/TestRewriteManifestsAction.java | 63 +++++++++++++++++++ 3 files changed, 89 insertions(+) diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java index b210cfcd4fa7..4f85151c80da 100644 --- a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -136,6 +136,23 @@ public static DeleteFile generatePositionDeleteFile(Table table, DataFile dataFi .build(); } + public static DeleteFile generatePositionDeleteFileWithRef(Table table, DataFile dataFile) { + PartitionSpec spec = table.specs().get(dataFile.specId()); + StructLike partition = dataFile.partition(); + LocationProvider locations = table.locationProvider(); + String path = locations.newDataLocation(spec, partition, generateFileName()); + long fileSize = generateFileSize(); + return FileMetadata.deleteFileBuilder(spec) + .ofPositionDeletes() + .withPath(path) + .withPartition(partition) + .withFileSizeInBytes(fileSize) + .withFormat(FileFormat.PARQUET) + .withReferencedDataFile(dataFile.location()) + .withRecordCount(3) + .build(); + } + // mimics the behavior of OutputFileFactory public static String generateFileName() { int partitionId = random().nextInt(100_000); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java index 99586f2503c2..af7c4a9b866d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java @@ -54,6 +54,7 @@ public abstract class SparkContentFile implements ContentFile { private final int sortOrderIdPosition; private final int fileSpecIdPosition; private final int equalityIdsPosition; + private final int referencedDataFilePosition; private final Type lowerBoundsType; private final Type upperBoundsType; private final Type keyMetadataType; @@ -103,6 +104,7 @@ public abstract class SparkContentFile implements ContentFile { this.sortOrderIdPosition = positions.get(DataFile.SORT_ORDER_ID.name()); this.fileSpecIdPosition = positions.get(DataFile.SPEC_ID.name()); this.equalityIdsPosition = positions.get(DataFile.EQUALITY_IDS.name()); + this.referencedDataFilePosition = positions.get(DataFile.REFERENCED_DATA_FILE.name()); } public F wrap(Row row) { @@ -231,6 +233,13 @@ public List equalityFieldIds() { return wrapped.isNullAt(equalityIdsPosition) ? null : wrapped.getList(equalityIdsPosition); } + public String referencedDataFile() { + if (wrapped.isNullAt(referencedDataFilePosition)) { + return null; + } + return wrapped.getString(referencedDataFilePosition); + } + private int fieldPosition(String name, StructType sparkType) { try { return sparkType.fieldIndex(name); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index a449de414a10..6cbc53baa349 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -41,7 +41,9 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Files; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; @@ -64,6 +66,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -128,6 +131,62 @@ public void setupTableLocation() throws Exception { this.tableLocation = tableDir.toURI().toString(); } + @TestTemplate + public void testRewriteManifestsPreservesOptionalFields() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + DataFile dataFile1 = newDataFile(table, "c1=0"); + DataFile dataFile2 = newDataFile(table, "c1=0"); + DataFile dataFile3 = newDataFile(table, "c1=0"); + table + .newFastAppend() + .appendFile(dataFile1) + .appendFile(dataFile2) + .appendFile(dataFile3) + .commit(); + + DeleteFile deleteFile1 = newDeleteFileWithRef(table, dataFile1); + assertThat(deleteFile1.referencedDataFile()).isEqualTo(dataFile1.location()); + table.newRowDelta().addDeletes(deleteFile1).commit(); + + DeleteFile deleteFile2 = newDeleteFileWithRef(table, dataFile2); + assertThat(deleteFile2.referencedDataFile()).isEqualTo(dataFile2.location()); + table.newRowDelta().addDeletes(deleteFile2).commit(); + + DeleteFile deleteFile3 = newDeleteFileWithRef(table, dataFile3); + assertThat(deleteFile3.referencedDataFile()).isEqualTo(dataFile3.location()); + table.newRowDelta().addDeletes(deleteFile3).commit(); + + SparkActions actions = SparkActions.get(); + + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + table.refresh(); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask fileTask : tasks) { + DataFile dataFile = fileTask.file(); + DeleteFile deleteFile = Iterables.getOnlyElement(fileTask.deletes()); + if (dataFile.location().equals(dataFile1.location())) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile1.referencedDataFile()); + } else if (dataFile.location().equals(dataFile2.location())) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile2.referencedDataFile()); + } else { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile3.referencedDataFile()); + } + } + } + } + @TestTemplate public void testRewriteManifestsEmptyTable() throws IOException { PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -976,6 +1035,10 @@ private DataFiles.Builder newDataFileBuilder(Table table) { .withRecordCount(1); } + private DeleteFile newDeleteFileWithRef(Table table, DataFile dataFile) { + return FileGenerationUtil.generatePositionDeleteFileWithRef(table, dataFile); + } + private DeleteFile newDeleteFile(Table table, String partitionPath) { return FileMetadata.deleteFileBuilder(table.spec()) .ofPositionDeletes() From 43b2f7d007c26ad79ebdf60d37ccca144db1f08f Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 5 Nov 2024 08:35:42 +0100 Subject: [PATCH 100/313] Core: Make PositionDeleteIndex serializable (#11463) --- .../deletes/BitmapPositionDeleteIndex.java | 124 ++++++++++++++++++ .../iceberg/deletes/PositionDeleteIndex.java | 21 +++ .../TestBitmapPositionDeleteIndex.java | 105 +++++++++++++++ .../all-container-types-position-index.bin | Bin 0 -> 94 bytes .../iceberg/deletes/empty-position-index.bin | Bin 0 -> 20 bytes ...mall-alternating-values-position-index.bin | Bin 0 -> 50 bytes .../small-and-large-values-position-index.bin | Bin 0 -> 56 bytes 7 files changed, 250 insertions(+) create mode 100644 core/src/test/resources/org/apache/iceberg/deletes/all-container-types-position-index.bin create mode 100644 core/src/test/resources/org/apache/iceberg/deletes/empty-position-index.bin create mode 100644 core/src/test/resources/org/apache/iceberg/deletes/small-alternating-values-position-index.bin create mode 100644 core/src/test/resources/org/apache/iceberg/deletes/small-and-large-values-position-index.bin diff --git a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java index cfb163e8379c..376b391d9c24 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java @@ -18,13 +18,23 @@ */ package org.apache.iceberg.deletes; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.Collection; import java.util.List; import java.util.function.LongConsumer; +import java.util.zip.CRC32; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; class BitmapPositionDeleteIndex implements PositionDeleteIndex { + private static final int LENGTH_SIZE_BYTES = 4; + private static final int MAGIC_NUMBER_SIZE_BYTES = 4; + private static final int CRC_SIZE_BYTES = 4; + private static final int BITMAP_DATA_OFFSET = 4; + private static final int MAGIC_NUMBER = 1681511377; + private final RoaringPositionBitmap bitmap; private final List deleteFiles; @@ -43,6 +53,11 @@ class BitmapPositionDeleteIndex implements PositionDeleteIndex { this.deleteFiles = deleteFile != null ? Lists.newArrayList(deleteFile) : Lists.newArrayList(); } + BitmapPositionDeleteIndex(RoaringPositionBitmap bitmap, DeleteFile deleteFile) { + this.bitmap = bitmap; + this.deleteFiles = deleteFile != null ? Lists.newArrayList(deleteFile) : Lists.newArrayList(); + } + void merge(BitmapPositionDeleteIndex that) { bitmap.setAll(that.bitmap); deleteFiles.addAll(that.deleteFiles); @@ -92,4 +107,113 @@ public Collection deleteFiles() { public long cardinality() { return bitmap.cardinality(); } + + /** + * Serializes the index using the following format: + * + *
      + *
    • The length of the magic bytes and bitmap stored as 4 bytes (big-endian). + *
    • A 4-byte {@link #MAGIC_NUMBER} (little-endian). + *
    • The bitmap serialized using the portable Roaring spec (little-endian). + *
    • A CRC-32 checksum of the magic bytes and bitmap as 4-bytes (big-endian). + *
    + * + * Note that the length and the checksum are computed for the bitmap data, which includes the + * magic bytes and bitmap for compatibility with Delta. + */ + @Override + public ByteBuffer serialize() { + bitmap.runLengthEncode(); // run-length encode the bitmap before serializing + int bitmapDataLength = computeBitmapDataLength(bitmap); // magic bytes + bitmap + byte[] bytes = new byte[LENGTH_SIZE_BYTES + bitmapDataLength + CRC_SIZE_BYTES]; + ByteBuffer buffer = ByteBuffer.wrap(bytes); + buffer.putInt(bitmapDataLength); + serializeBitmapData(bytes, bitmapDataLength, bitmap); + int crcOffset = LENGTH_SIZE_BYTES + bitmapDataLength; + int crc = computeChecksum(bytes, bitmapDataLength); + buffer.putInt(crcOffset, crc); + buffer.rewind(); + return buffer; + } + + /** + * Deserializes the index from bytes, assuming the format described in {@link #serialize()}. + * + * @param bytes an array containing the serialized index + * @param deleteFile the DV file + * @return the deserialized index + */ + public static PositionDeleteIndex deserialize(byte[] bytes, DeleteFile deleteFile) { + ByteBuffer buffer = ByteBuffer.wrap(bytes); + int bitmapDataLength = readBitmapDataLength(buffer, deleteFile); + RoaringPositionBitmap bitmap = deserializeBitmap(bytes, bitmapDataLength, deleteFile); + int crc = computeChecksum(bytes, bitmapDataLength); + int crcOffset = LENGTH_SIZE_BYTES + bitmapDataLength; + int expectedCrc = buffer.getInt(crcOffset); + Preconditions.checkArgument(crc == expectedCrc, "Invalid CRC"); + return new BitmapPositionDeleteIndex(bitmap, deleteFile); + } + + // computes and validates the length of the bitmap data (magic bytes + bitmap) + private static int computeBitmapDataLength(RoaringPositionBitmap bitmap) { + long length = MAGIC_NUMBER_SIZE_BYTES + bitmap.serializedSizeInBytes(); + long bufferSize = LENGTH_SIZE_BYTES + length + CRC_SIZE_BYTES; + Preconditions.checkState(bufferSize <= Integer.MAX_VALUE, "Can't serialize index > 2GB"); + return (int) length; + } + + // serializes the bitmap data (magic bytes + bitmap) using the little-endian byte order + private static void serializeBitmapData( + byte[] bytes, int bitmapDataLength, RoaringPositionBitmap bitmap) { + ByteBuffer bitmapData = pointToBitmapData(bytes, bitmapDataLength); + bitmapData.putInt(MAGIC_NUMBER); + bitmap.serialize(bitmapData); + } + + // points to the bitmap data in the blob + private static ByteBuffer pointToBitmapData(byte[] bytes, int bitmapDataLength) { + ByteBuffer bitmapData = ByteBuffer.wrap(bytes, BITMAP_DATA_OFFSET, bitmapDataLength); + bitmapData.order(ByteOrder.LITTLE_ENDIAN); + return bitmapData; + } + + // checks the blob size is equal to the bitmap data length + extra bytes for length and CRC + private static int readBitmapDataLength(ByteBuffer buffer, DeleteFile deleteFile) { + int length = buffer.getInt(); + long expectedLength = deleteFile.contentSizeInBytes() - LENGTH_SIZE_BYTES - CRC_SIZE_BYTES; + Preconditions.checkArgument( + length == expectedLength, + "Invalid bitmap data length: %s, expected %s", + length, + expectedLength); + return length; + } + + // validates magic bytes and deserializes the bitmap + private static RoaringPositionBitmap deserializeBitmap( + byte[] bytes, int bitmapDataLength, DeleteFile deleteFile) { + ByteBuffer bitmapData = pointToBitmapData(bytes, bitmapDataLength); + int magicNumber = bitmapData.getInt(); + Preconditions.checkArgument( + magicNumber == MAGIC_NUMBER, + "Invalid magic number: %s, expected %s", + magicNumber, + MAGIC_NUMBER); + RoaringPositionBitmap bitmap = RoaringPositionBitmap.deserialize(bitmapData); + long cardinality = bitmap.cardinality(); + long expectedCardinality = deleteFile.recordCount(); + Preconditions.checkArgument( + cardinality == expectedCardinality, + "Invalid cardinality: %s, expected %s", + cardinality, + expectedCardinality); + return bitmap; + } + + // generates a 32-bit unsigned checksum for the magic bytes and serialized bitmap + private static int computeChecksum(byte[] bytes, int bitmapDataLength) { + CRC32 crc = new CRC32(); + crc.update(bytes, BITMAP_DATA_OFFSET, bitmapDataLength); + return (int) crc.getValue(); + } } diff --git a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java index 8ccfc03d1a26..6f97b3a6ac87 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.deletes; +import java.nio.ByteBuffer; import java.util.Collection; import java.util.function.LongConsumer; import org.apache.iceberg.DeleteFile; @@ -92,6 +93,26 @@ default long cardinality() { throw new UnsupportedOperationException(getClass().getName() + " does not support cardinality"); } + /** + * Serializes this index. + * + * @return a buffer containing the serialized index + */ + default ByteBuffer serialize() { + throw new UnsupportedOperationException(getClass().getName() + " does not support serialize"); + } + + /** + * Deserializes a position delete index. + * + * @param bytes an array containing the serialized index + * @param deleteFile the delete file that the index is created for + * @return the deserialized index + */ + static PositionDeleteIndex deserialize(byte[] bytes, DeleteFile deleteFile) { + return BitmapPositionDeleteIndex.deserialize(bytes, deleteFile); + } + /** Returns an empty immutable position delete index. */ static PositionDeleteIndex empty() { return EmptyPositionDeleteIndex.get(); diff --git a/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java b/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java index c8fc723deb9e..76b294f80611 100644 --- a/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java +++ b/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java @@ -20,12 +20,21 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.io.IOException; +import java.net.URL; +import java.nio.ByteBuffer; import java.util.List; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.io.Resources; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; public class TestBitmapPositionDeleteIndex { + private static final long BITMAP_OFFSET = 0xFFFFFFFFL + 1L; + private static final long CONTAINER_OFFSET = Character.MAX_VALUE + 1L; + @Test public void testForEach() { long pos1 = 10L; // Container 0 (high bits = 0) @@ -105,6 +114,102 @@ public void testMergeBitmapIndexWithEmpty() { assertThat(positions).containsExactly(pos1, pos2, pos3, pos4); } + @Test + public void testEmptyIndexSerialization() throws Exception { + PositionDeleteIndex index = new BitmapPositionDeleteIndex(); + validate(index, "empty-position-index.bin"); + } + + @Test + public void testSmallAlternatingValuesIndexSerialization() throws Exception { + PositionDeleteIndex index = new BitmapPositionDeleteIndex(); + index.delete(1L); + index.delete(3L); + index.delete(5L); + index.delete(7L); + index.delete(9L); + validate(index, "small-alternating-values-position-index.bin"); + } + + @Test + public void testSmallAndLargeValuesIndexSerialization() throws Exception { + PositionDeleteIndex index = new BitmapPositionDeleteIndex(); + index.delete(100L); + index.delete(101L); + index.delete(Integer.MAX_VALUE + 100L); + index.delete(Integer.MAX_VALUE + 101L); + validate(index, "small-and-large-values-position-index.bin"); + } + + @Test + public void testAllContainerTypesIndexSerialization() throws Exception { + PositionDeleteIndex index = new BitmapPositionDeleteIndex(); + + // bitmap 0, container 0 (array) + index.delete(position(0 /* bitmap */, 0 /* container */, 5L)); + index.delete(position(0 /* bitmap */, 0 /* container */, 7L)); + + // bitmap 0, container 1 (array that can be compressed) + index.delete( + position(0 /* bitmap */, 1 /* container */, 1L), + position(0 /* bitmap */, 1 /* container */, 1000L)); + + // bitmap 1, container 2 (bitset) + index.delete( + position(0 /* bitmap */, 2 /* container */, 1L), + position(0 /* bitmap */, 2 /* container */, CONTAINER_OFFSET - 1L)); + + // bitmap 1, container 0 (array) + index.delete(position(1 /* bitmap */, 0 /* container */, 10L)); + index.delete(position(1 /* bitmap */, 0 /* container */, 20L)); + + // bitmap 1, container 1 (array that can be compressed) + index.delete( + position(1 /* bitmap */, 1 /* container */, 10L), + position(1 /* bitmap */, 1 /* container */, 500L)); + + // bitmap 1, container 2 (bitset) + index.delete( + position(1 /* bitmap */, 2 /* container */, 1L), + position(1 /* bitmap */, 2 /* container */, CONTAINER_OFFSET - 1)); + + validate(index, "all-container-types-position-index.bin"); + } + + private static void validate(PositionDeleteIndex index, String goldenFile) throws Exception { + ByteBuffer buffer = index.serialize(); + byte[] bytes = buffer.array(); + DeleteFile dv = mockDV(bytes.length, index.cardinality()); + PositionDeleteIndex indexCopy = PositionDeleteIndex.deserialize(bytes, dv); + assertEqual(index, indexCopy); + byte[] goldenBytes = readTestResource(goldenFile); + assertThat(bytes).isEqualTo(goldenBytes); + PositionDeleteIndex goldenIndex = PositionDeleteIndex.deserialize(goldenBytes, dv); + assertEqual(index, goldenIndex); + } + + private static DeleteFile mockDV(long contentSize, long cardinality) { + DeleteFile mock = Mockito.mock(DeleteFile.class); + Mockito.when(mock.contentSizeInBytes()).thenReturn(contentSize); + Mockito.when(mock.recordCount()).thenReturn(cardinality); + return mock; + } + + private static void assertEqual(PositionDeleteIndex index, PositionDeleteIndex thatIndex) { + assertThat(index.cardinality()).isEqualTo(thatIndex.cardinality()); + index.forEach(position -> assertThat(thatIndex.isDeleted(position)).isTrue()); + thatIndex.forEach(position -> assertThat(index.isDeleted(position)).isTrue()); + } + + private static long position(int bitmapIndex, int containerIndex, long value) { + return bitmapIndex * BITMAP_OFFSET + containerIndex * CONTAINER_OFFSET + value; + } + + private static byte[] readTestResource(String resourceName) throws IOException { + URL resource = Resources.getResource(TestRoaringPositionBitmap.class, resourceName); + return Resources.toByteArray(resource); + } + private List collect(PositionDeleteIndex index) { List positions = Lists.newArrayList(); index.forEach(positions::add); diff --git a/core/src/test/resources/org/apache/iceberg/deletes/all-container-types-position-index.bin b/core/src/test/resources/org/apache/iceberg/deletes/all-container-types-position-index.bin new file mode 100644 index 0000000000000000000000000000000000000000..00d47303b11b4dcf167754039cbd7753e770f2c3 GIT binary patch literal 94 zcmZQzUAd?7#M-@88Z{Z-~X%(>`)#^?C*a@pjsqFFBw6KxEMr$ Ql3WZg86k>J%ziu_0Iw$!Hvj+t literal 0 HcmV?d00001 diff --git a/core/src/test/resources/org/apache/iceberg/deletes/empty-position-index.bin b/core/src/test/resources/org/apache/iceberg/deletes/empty-position-index.bin new file mode 100644 index 0000000000000000000000000000000000000000..8bbc1265dc1dfbce43c8074e8f21fce1f8c8c8db GIT binary patch literal 20 WcmZQzVBooU*)oLz3ieBQ@BjcNUjumn literal 0 HcmV?d00001 diff --git a/core/src/test/resources/org/apache/iceberg/deletes/small-alternating-values-position-index.bin b/core/src/test/resources/org/apache/iceberg/deletes/small-alternating-values-position-index.bin new file mode 100644 index 0000000000000000000000000000000000000000..80829fae22c3f39f2d1b6e3bab0a4222ca13c433 GIT binary patch literal 50 rcmZQzV9>gF*)oNZ0RgNG7#JXY76t*JFe3vq11keN11H1g-wOKxf}#bH literal 0 HcmV?d00001 diff --git a/core/src/test/resources/org/apache/iceberg/deletes/small-and-large-values-position-index.bin b/core/src/test/resources/org/apache/iceberg/deletes/small-and-large-values-position-index.bin new file mode 100644 index 0000000000000000000000000000000000000000..989dabf6ad53528a2ffe1a66240dde713653ce30 GIT binary patch literal 56 ucmZQzU@*9N*)oNZ0RgNG7#NtqT%c$JBZCBxEd#_U45 Date: Tue, 5 Nov 2024 08:50:39 +0100 Subject: [PATCH 101/313] Spark 3.5: Preserve content offset and size during manifest rewrites (#11469) --- .../iceberg/spark/SparkContentFile.java | 18 +++++++ .../actions/TestRewriteManifestsAction.java | 48 ++++++++++++++++--- 2 files changed, 59 insertions(+), 7 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java index af7c4a9b866d..bad31d8d85f4 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java @@ -55,6 +55,8 @@ public abstract class SparkContentFile implements ContentFile { private final int fileSpecIdPosition; private final int equalityIdsPosition; private final int referencedDataFilePosition; + private final int contentOffsetPosition; + private final int contentSizePosition; private final Type lowerBoundsType; private final Type upperBoundsType; private final Type keyMetadataType; @@ -105,6 +107,8 @@ public abstract class SparkContentFile implements ContentFile { this.fileSpecIdPosition = positions.get(DataFile.SPEC_ID.name()); this.equalityIdsPosition = positions.get(DataFile.EQUALITY_IDS.name()); this.referencedDataFilePosition = positions.get(DataFile.REFERENCED_DATA_FILE.name()); + this.contentOffsetPosition = positions.get(DataFile.CONTENT_OFFSET.name()); + this.contentSizePosition = positions.get(DataFile.CONTENT_SIZE.name()); } public F wrap(Row row) { @@ -240,6 +244,20 @@ public String referencedDataFile() { return wrapped.getString(referencedDataFilePosition); } + public Long contentOffset() { + if (wrapped.isNullAt(contentOffsetPosition)) { + return null; + } + return wrapped.getLong(contentOffsetPosition); + } + + public Long contentSizeInBytes() { + if (wrapped.isNullAt(contentSizePosition)) { + return null; + } + return wrapped.getLong(contentSizePosition); + } + private int fieldPosition(String name, StructType sparkType) { try { return sparkType.fieldIndex(name); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 6cbc53baa349..b86d74415ac8 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -106,7 +106,8 @@ public static Object[] parameters() { new Object[] {"true", "true", false, 1}, new Object[] {"false", "true", true, 1}, new Object[] {"true", "false", false, 2}, - new Object[] {"false", "false", false, 2} + new Object[] {"false", "false", false, 2}, + new Object[] {"false", "false", false, 3} }; } @@ -150,16 +151,16 @@ public void testRewriteManifestsPreservesOptionalFields() throws IOException { .appendFile(dataFile3) .commit(); - DeleteFile deleteFile1 = newDeleteFileWithRef(table, dataFile1); - assertThat(deleteFile1.referencedDataFile()).isEqualTo(dataFile1.location()); + DeleteFile deleteFile1 = newDeletes(table, dataFile1); + assertDeletes(dataFile1, deleteFile1); table.newRowDelta().addDeletes(deleteFile1).commit(); - DeleteFile deleteFile2 = newDeleteFileWithRef(table, dataFile2); - assertThat(deleteFile2.referencedDataFile()).isEqualTo(dataFile2.location()); + DeleteFile deleteFile2 = newDeletes(table, dataFile2); + assertDeletes(dataFile2, deleteFile2); table.newRowDelta().addDeletes(deleteFile2).commit(); - DeleteFile deleteFile3 = newDeleteFileWithRef(table, dataFile3); - assertThat(deleteFile3.referencedDataFile()).isEqualTo(dataFile3.location()); + DeleteFile deleteFile3 = newDeletes(table, dataFile3); + assertDeletes(dataFile3, deleteFile3); table.newRowDelta().addDeletes(deleteFile3).commit(); SparkActions actions = SparkActions.get(); @@ -178,10 +179,13 @@ public void testRewriteManifestsPreservesOptionalFields() throws IOException { DeleteFile deleteFile = Iterables.getOnlyElement(fileTask.deletes()); if (dataFile.location().equals(dataFile1.location())) { assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile1.referencedDataFile()); + assertEqual(deleteFile, deleteFile1); } else if (dataFile.location().equals(dataFile2.location())) { assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile2.referencedDataFile()); + assertEqual(deleteFile, deleteFile2); } else { assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile3.referencedDataFile()); + assertEqual(deleteFile, deleteFile3); } } } @@ -1035,10 +1039,18 @@ private DataFiles.Builder newDataFileBuilder(Table table) { .withRecordCount(1); } + private DeleteFile newDeletes(Table table, DataFile dataFile) { + return formatVersion >= 3 ? newDV(table, dataFile) : newDeleteFileWithRef(table, dataFile); + } + private DeleteFile newDeleteFileWithRef(Table table, DataFile dataFile) { return FileGenerationUtil.generatePositionDeleteFileWithRef(table, dataFile); } + private DeleteFile newDV(Table table, DataFile dataFile) { + return FileGenerationUtil.generateDV(table, dataFile); + } + private DeleteFile newDeleteFile(Table table, String partitionPath) { return FileMetadata.deleteFileBuilder(table.spec()) .ofPositionDeletes() @@ -1097,4 +1109,26 @@ private DeleteFile writeEqDeletes(Table table, StructLike partition, String key, OutputFile outputFile = Files.localOutput(File.createTempFile("junit", null, temp.toFile())); return FileHelpers.writeDeleteFile(table, outputFile, partition, deletes, deleteSchema); } + + private void assertDeletes(DataFile dataFile, DeleteFile deleteFile) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(dataFile.location()); + if (formatVersion >= 3) { + assertThat(deleteFile.contentOffset()).isNotNull(); + assertThat(deleteFile.contentSizeInBytes()).isNotNull(); + } else { + assertThat(deleteFile.contentOffset()).isNull(); + assertThat(deleteFile.contentSizeInBytes()).isNull(); + } + } + + private void assertEqual(DeleteFile deleteFile1, DeleteFile deleteFile2) { + assertThat(deleteFile1.location()).isEqualTo(deleteFile2.location()); + assertThat(deleteFile1.content()).isEqualTo(deleteFile2.content()); + assertThat(deleteFile1.specId()).isEqualTo(deleteFile2.specId()); + assertThat(deleteFile1.partition()).isEqualTo(deleteFile2.partition()); + assertThat(deleteFile1.format()).isEqualTo(deleteFile2.format()); + assertThat(deleteFile1.referencedDataFile()).isEqualTo(deleteFile2.referencedDataFile()); + assertThat(deleteFile1.contentOffset()).isEqualTo(deleteFile2.contentOffset()); + assertThat(deleteFile1.contentSizeInBytes()).isEqualTo(deleteFile2.contentSizeInBytes()); + } } From 20e0e3dae59826ea02bc2c79f8778cdf7a825316 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 5 Nov 2024 18:04:56 +0800 Subject: [PATCH 102/313] Spark 3.5: Fix flaky test due to temp directory not empty during delete (#11470) --- .../spark/source/TestDataFrameWrites.java | 67 +++++++------------ 1 file changed, 23 insertions(+), 44 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 01c36b824ea6..a8b4c915868a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -28,7 +28,6 @@ import java.io.File; import java.io.IOException; import java.net.URI; -import java.nio.file.NoSuchFileException; import java.util.Arrays; import java.util.Collection; import java.util.Iterator; @@ -36,7 +35,6 @@ import java.util.Map; import java.util.Random; import org.apache.avro.generic.GenericData.Record; -import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.Files; import org.apache.iceberg.Parameter; @@ -76,6 +74,7 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; @ExtendWith(ParameterizedTestExtension.class) public class TestDataFrameWrites extends ParameterizedAvroDataTest { @@ -88,6 +87,8 @@ public static Collection parameters() { @Parameter private String format; + @TempDir private File location; + private static SparkSession spark = null; private static JavaSparkContext sc = null; @@ -140,47 +141,37 @@ public static void stopSpark() { @Override protected void writeAndValidate(Schema schema) throws IOException { - File location = createTableFolder(); - Table table = createTable(schema, location); - writeAndValidateWithLocations(table, location, new File(location, "data")); + Table table = createTable(schema); + writeAndValidateWithLocations(table, new File(location, "data")); } @TestTemplate public void testWriteWithCustomDataLocation() throws IOException { - File location = createTableFolder(); File tablePropertyDataLocation = temp.resolve("test-table-property-data-dir").toFile(); - Table table = createTable(new Schema(SUPPORTED_PRIMITIVES.fields()), location); + Table table = createTable(new Schema(SUPPORTED_PRIMITIVES.fields())); table .updateProperties() .set(TableProperties.WRITE_DATA_LOCATION, tablePropertyDataLocation.getAbsolutePath()) .commit(); - writeAndValidateWithLocations(table, location, tablePropertyDataLocation); - } - - private File createTableFolder() throws IOException { - File parent = temp.resolve("parquet").toFile(); - File location = new File(parent, "test"); - assertThat(location.mkdirs()).as("Mkdir should succeed").isTrue(); - return location; + writeAndValidateWithLocations(table, tablePropertyDataLocation); } - private Table createTable(Schema schema, File location) { + private Table createTable(Schema schema) { HadoopTables tables = new HadoopTables(CONF); return tables.create(schema, PartitionSpec.unpartitioned(), location.toString()); } - private void writeAndValidateWithLocations(Table table, File location, File expectedDataDir) - throws IOException { + private void writeAndValidateWithLocations(Table table, File expectedDataDir) throws IOException { Schema tableSchema = table.schema(); // use the table schema because ids are reassigned table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit(); Iterable expected = RandomData.generate(tableSchema, 100, 0L); - writeData(expected, tableSchema, location.toString()); + writeData(expected, tableSchema); table.refresh(); - List actual = readTable(location.toString()); + List actual = readTable(); Iterator expectedIter = expected.iterator(); Iterator actualIter = actual.iterator(); @@ -204,21 +195,20 @@ private void writeAndValidateWithLocations(Table table, File location, File expe .startsWith(expectedDataDir.getAbsolutePath())); } - private List readTable(String location) { - Dataset result = spark.read().format("iceberg").load(location); + private List readTable() { + Dataset result = spark.read().format("iceberg").load(location.toString()); return result.collectAsList(); } - private void writeData(Iterable records, Schema schema, String location) - throws IOException { + private void writeData(Iterable records, Schema schema) throws IOException { Dataset df = createDataset(records, schema); DataFrameWriter writer = df.write().format("iceberg").mode("append"); - writer.save(location); + writer.save(location.toString()); } - private void writeDataWithFailOnPartition( - Iterable records, Schema schema, String location) throws IOException, SparkException { + private void writeDataWithFailOnPartition(Iterable records, Schema schema) + throws IOException, SparkException { final int numPartitions = 10; final int partitionToFail = new Random().nextInt(numPartitions); MapPartitionsFunction failOnFirstPartitionFunc = @@ -241,7 +231,7 @@ private void writeDataWithFailOnPartition( // Setting "check-nullability" option to "false" doesn't help as it fails at Spark analyzer. Dataset convertedDf = df.sqlContext().createDataFrame(df.rdd(), convert(schema)); DataFrameWriter writer = convertedDf.write().format("iceberg").mode("append"); - writer.save(location); + writer.save(location.toString()); } private Dataset createDataset(Iterable records, Schema schema) throws IOException { @@ -287,7 +277,6 @@ public void testNullableWithWriteOption() throws IOException { .as("Spark 3 rejects writing nulls to a required column") .startsWith("2"); - File location = temp.resolve("parquet").resolve("test").toFile(); String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location); String targetPath = String.format("%s/nullable_poc/targetFolder/", location); @@ -341,7 +330,6 @@ public void testNullableWithSparkSqlOption() throws IOException { .as("Spark 3 rejects writing nulls to a required column") .startsWith("2"); - File location = temp.resolve("parquet").resolve("test").toFile(); String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location); String targetPath = String.format("%s/nullable_poc/targetFolder/", location); @@ -397,37 +385,28 @@ public void testNullableWithSparkSqlOption() throws IOException { @TestTemplate public void testFaultToleranceOnWrite() throws IOException { - File location = createTableFolder(); Schema schema = new Schema(SUPPORTED_PRIMITIVES.fields()); - Table table = createTable(schema, location); + Table table = createTable(schema); Iterable records = RandomData.generate(schema, 100, 0L); - writeData(records, schema, location.toString()); + writeData(records, schema); table.refresh(); Snapshot snapshotBeforeFailingWrite = table.currentSnapshot(); - List resultBeforeFailingWrite = readTable(location.toString()); + List resultBeforeFailingWrite = readTable(); Iterable records2 = RandomData.generate(schema, 100, 0L); - assertThatThrownBy(() -> writeDataWithFailOnPartition(records2, schema, location.toString())) + assertThatThrownBy(() -> writeDataWithFailOnPartition(records2, schema)) .isInstanceOf(SparkException.class); table.refresh(); Snapshot snapshotAfterFailingWrite = table.currentSnapshot(); - List resultAfterFailingWrite = readTable(location.toString()); + List resultAfterFailingWrite = readTable(); assertThat(snapshotBeforeFailingWrite).isEqualTo(snapshotAfterFailingWrite); assertThat(resultBeforeFailingWrite).isEqualTo(resultAfterFailingWrite); - - while (location.exists()) { - try { - FileUtils.deleteDirectory(location); - } catch (NoSuchFileException e) { - // ignore NoSuchFileException when a file is already deleted - } - } } } From 67ee0825959c42b405c2cc0e2e5b916bd2cfc493 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 5 Nov 2024 12:56:02 +0100 Subject: [PATCH 103/313] Core, Data, Flink, Spark: Improve tableDir initialization for tests (#11460) --- .../iceberg/DeleteFileIndexTestBase.java | 5 +--- .../apache/iceberg/FilterFilesTestBase.java | 10 +------ .../java/org/apache/iceberg/ScanTestBase.java | 15 +++++----- .../apache/iceberg/TestCreateTransaction.java | 29 ------------------- .../iceberg/TestMetadataTableScans.java | 2 -- ...adataTableScansWithPartitionEvolution.java | 4 --- .../java/org/apache/iceberg/TestMetrics.java | 10 ++----- .../org/apache/iceberg/TestMetricsModes.java | 11 +------ .../org/apache/iceberg/TestOverwrite.java | 5 ---- .../iceberg/TestOverwriteWithValidation.java | 4 --- ...stPartitionSpecBuilderCaseSensitivity.java | 14 --------- .../apache/iceberg/TestPartitionSpecInfo.java | 12 +------- .../org/apache/iceberg/TestPartitioning.java | 12 +------- .../apache/iceberg/TestReplacePartitions.java | 11 ------- .../iceberg/TestReplaceTransaction.java | 7 ----- .../org/apache/iceberg/TestSortOrder.java | 13 +-------- .../org/apache/iceberg/TestSplitPlanning.java | 3 -- .../iceberg/TestTimestampPartitions.java | 5 ---- .../apache/iceberg/util/TestSnapshotUtil.java | 2 -- .../org/apache/iceberg/TestSplitScan.java | 6 ++-- .../data/TestGenericReaderDeletes.java | 8 ++--- .../iceberg/io/TestAppenderFactory.java | 4 --- .../apache/iceberg/io/TestBaseTaskWriter.java | 3 -- .../iceberg/io/TestFileWriterFactory.java | 4 --- .../iceberg/io/TestPartitioningWriters.java | 4 --- .../iceberg/io/TestPositionDeltaWriters.java | 4 --- .../iceberg/io/TestRollingFileWriters.java | 4 --- .../io/TestTaskEqualityDeltaWriter.java | 3 -- .../apache/iceberg/io/TestWriterMetrics.java | 12 +------- .../flink/sink/TestDeltaTaskWriter.java | 3 -- .../flink/sink/TestIcebergFilesCommitter.java | 3 -- .../source/TestStreamingMonitorFunction.java | 3 -- .../source/TestStreamingReaderOperator.java | 3 -- .../flink/sink/TestDeltaTaskWriter.java | 3 -- .../flink/sink/TestIcebergFilesCommitter.java | 3 -- .../source/TestStreamingMonitorFunction.java | 3 -- .../source/TestStreamingReaderOperator.java | 3 -- .../flink/sink/TestDeltaTaskWriter.java | 3 -- .../flink/sink/TestIcebergFilesCommitter.java | 3 -- .../source/TestStreamingMonitorFunction.java | 3 -- .../source/TestStreamingReaderOperator.java | 3 -- .../iceberg/TestScanTaskSerialization.java | 2 +- .../spark/actions/TestCreateActions.java | 8 ++--- .../TestDeleteReachableFilesAction.java | 4 +-- .../actions/TestExpireSnapshotsAction.java | 3 +- .../TestRemoveDanglingDeleteAction.java | 4 +-- .../actions/TestRemoveOrphanFilesAction.java | 4 +-- .../actions/TestRewriteDataFilesAction.java | 2 +- .../actions/TestRewriteManifestsAction.java | 2 +- .../source/TestIcebergSourceHadoopTables.java | 7 ++--- .../spark/source/TestSparkDataFile.java | 4 +-- 51 files changed, 32 insertions(+), 270 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java index 986e8608c082..de7e59ac170d 100644 --- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java @@ -266,11 +266,8 @@ public void testPartitionedDeleteIndex() { @TestTemplate public void testUnpartitionedTableScan() throws IOException { - File location = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(location.delete()).isTrue(); - Table unpartitioned = - TestTables.create(location, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), 2); + TestTables.create(tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), 2); DataFile unpartitionedFile = unpartitionedFile(unpartitioned.spec()); unpartitioned.newAppend().appendFile(unpartitionedFile).commit(); diff --git a/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java b/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java index bb4bb282a330..4bdffffa3028 100644 --- a/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java +++ b/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java @@ -22,9 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.File; -import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.file.Files; import java.nio.file.Path; import java.util.Map; import org.apache.iceberg.expressions.Expressions; @@ -32,7 +30,6 @@ import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; @@ -50,12 +47,7 @@ public abstract class FilterFilesTestBase< private final Schema schema = new Schema( required(1, "id", Types.IntegerType.get()), required(2, "data", Types.StringType.get())); - private File tableDir = null; - - @BeforeEach - public void setupTableDir() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - } + @TempDir private File tableDir; @AfterEach public void cleanupTables() { diff --git a/core/src/test/java/org/apache/iceberg/ScanTestBase.java b/core/src/test/java/org/apache/iceberg/ScanTestBase.java index 51d692c1b44d..de1173033742 100644 --- a/core/src/test/java/org/apache/iceberg/ScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanTestBase.java @@ -23,9 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; -import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Collections; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; @@ -149,9 +147,8 @@ public void testReAddingPartitionField() throws Exception { required(2, "b", Types.StringType.get()), required(3, "data", Types.IntegerType.get())); PartitionSpec initialSpec = PartitionSpec.builderFor(schema).identity("a").build(); - File dir = Files.createTempDirectory(temp, "junit").toFile(); - dir.delete(); - this.table = TestTables.create(dir, "test_part_evolution", schema, initialSpec, formatVersion); + this.table = + TestTables.create(tableDir, "test_part_evolution", schema, initialSpec, formatVersion); table .newFastAppend() .appendFile( @@ -222,11 +219,13 @@ public void testDataFileSorted() throws Exception { Schema schema = new Schema( required(1, "a", Types.IntegerType.get()), required(2, "b", Types.StringType.get())); - File dir = Files.createTempDirectory(temp, "junit").toFile(); - dir.delete(); this.table = TestTables.create( - dir, "test_data_file_sorted", schema, PartitionSpec.unpartitioned(), formatVersion); + tableDir, + "test_data_file_sorted", + schema, + PartitionSpec.unpartitioned(), + formatVersion); table .newFastAppend() .appendFile( diff --git a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java index 766dd85a0655..667c4b8cc133 100644 --- a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java @@ -23,9 +23,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import org.apache.iceberg.exceptions.CommitFailedException; @@ -45,9 +43,6 @@ protected static List parameters() { @TestTemplate public void testCreateTransaction() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Transaction txn = TestTables.beginCreate(tableDir, "test_create", SCHEMA, unpartitioned()); assertThat(TestTables.readMetadata("test_create")).isNull(); @@ -68,9 +63,6 @@ public void testCreateTransaction() throws IOException { @TestTemplate public void testCreateTransactionAndUpdateSchema() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Transaction txn = TestTables.beginCreate(tableDir, "test_create", SCHEMA, unpartitioned()); assertThat(TestTables.readMetadata("test_create")).isNull(); @@ -105,9 +97,6 @@ public void testCreateTransactionAndUpdateSchema() throws IOException { @TestTemplate public void testCreateAndAppendWithTransaction() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Transaction txn = TestTables.beginCreate(tableDir, "test_append", SCHEMA, unpartitioned()); assertThat(TestTables.readMetadata("test_append")).isNull(); @@ -134,9 +123,6 @@ public void testCreateAndAppendWithTransaction() throws IOException { @TestTemplate public void testCreateAndAppendWithTable() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Transaction txn = TestTables.beginCreate(tableDir, "test_append", SCHEMA, unpartitioned()); assertThat(TestTables.readMetadata("test_append")) @@ -167,9 +153,6 @@ public void testCreateAndAppendWithTable() throws IOException { @TestTemplate public void testCreateAndUpdatePropertiesWithTransaction() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Transaction txn = TestTables.beginCreate(tableDir, "test_properties", SCHEMA, unpartitioned()); assertThat(TestTables.readMetadata("test_properties")).isNull(); @@ -196,9 +179,6 @@ public void testCreateAndUpdatePropertiesWithTransaction() throws IOException { @TestTemplate public void testCreateAndUpdatePropertiesWithTable() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Transaction txn = TestTables.beginCreate(tableDir, "test_properties", SCHEMA, unpartitioned()); assertThat(TestTables.readMetadata("test_properties")).isNull(); @@ -227,9 +207,6 @@ public void testCreateAndUpdatePropertiesWithTable() throws IOException { @TestTemplate public void testCreateDetectsUncommittedChange() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Transaction txn = TestTables.beginCreate(tableDir, "uncommitted_change", SCHEMA, unpartitioned()); @@ -245,9 +222,6 @@ public void testCreateDetectsUncommittedChange() throws IOException { @TestTemplate public void testCreateDetectsUncommittedChangeOnCommit() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Transaction txn = TestTables.beginCreate(tableDir, "uncommitted_change", SCHEMA, unpartitioned()); @@ -263,9 +237,6 @@ public void testCreateDetectsUncommittedChangeOnCommit() throws IOException { @TestTemplate public void testCreateTransactionConflict() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Transaction txn = TestTables.beginCreate(tableDir, "test_conflict", SCHEMA, SPEC); // append in the transaction to ensure a manifest file is created diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 146b859bef00..30fdae01cd94 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -25,7 +25,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -1066,7 +1065,6 @@ public void testPartitionSpecEvolutionRemoval() { @TestTemplate public void testPartitionColumnNamedPartition() throws Exception { TestTables.clearTables(); - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Schema schema = new Schema( diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java index ac96642319a3..2de38541777b 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java @@ -25,7 +25,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.List; import java.util.Map; import java.util.stream.Stream; @@ -45,9 +44,6 @@ public class TestMetadataTableScansWithPartitionEvolution extends MetadataTableS @BeforeEach public void createTable() throws IOException { TestTables.clearTables(); - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - tableDir.delete(); - Schema schema = new Schema( required(1, "id", Types.IntegerType.get()), diff --git a/core/src/test/java/org/apache/iceberg/TestMetrics.java b/core/src/test/java/org/apache/iceberg/TestMetrics.java index 2c4849135f64..0099f93305ea 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetrics.java +++ b/core/src/test/java/org/apache/iceberg/TestMetrics.java @@ -30,7 +30,6 @@ import java.nio.ByteBuffer; import java.nio.CharBuffer; import java.nio.charset.StandardCharsets; -import java.nio.file.Files; import java.nio.file.Path; import java.util.Arrays; import java.util.List; @@ -72,7 +71,8 @@ public static List parameters() { return Arrays.asList(1, 2, 3); } - @TempDir public Path temp; + @TempDir protected Path temp; + @TempDir private File tableDir; private static final StructType LEAF_STRUCT_TYPE = StructType.of( @@ -676,9 +676,6 @@ public void testTruncateBinaryMetricsMode() throws IOException { @TestTemplate public void testSortedColumnMetrics() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created by table create - SortOrder sortOrder = SortOrder.builderFor(SIMPLE_SCHEMA) .asc("booleanCol") @@ -739,9 +736,6 @@ public void testSortedColumnMetrics() throws IOException { @TestTemplate public void testMetricsForSortedNestedStructFields() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created by table create - SortOrder sortOrder = SortOrder.builderFor(NESTED_SCHEMA) .asc("nestedStructCol.longCol") diff --git a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java index 00e961097c34..f766bb69a194 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java +++ b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java @@ -24,7 +24,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.nio.file.Path; import java.util.Arrays; import java.util.List; @@ -42,6 +41,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestMetricsModes { + @TempDir private File tableDir; @Parameter private int formatVersion; @@ -108,9 +108,6 @@ public void testInvalidDefaultColumnModeValue() { @TestTemplate public void testMetricsConfigSortedColsDefault() throws Exception { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Schema schema = new Schema( required(1, "col1", Types.IntegerType.get()), @@ -145,9 +142,6 @@ public void testMetricsConfigSortedColsDefault() throws Exception { @TestTemplate public void testMetricsConfigSortedColsDefaultByInvalid() throws Exception { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Schema schema = new Schema( required(1, "col1", Types.IntegerType.get()), @@ -181,9 +175,6 @@ public void testMetricsConfigInferredDefaultModeLimit() throws IOException { required(2, "col2", Types.IntegerType.get()), required(3, "col3", Types.IntegerType.get())); - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Table table = TestTables.create( tableDir, diff --git a/core/src/test/java/org/apache/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java index d4c886cefc4e..a490350159bf 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -27,11 +27,9 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import org.apache.iceberg.ManifestEntry.Status; @@ -126,9 +124,6 @@ private static ByteBuffer longToBuffer(long value) { @BeforeEach public void createTestTable() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - this.table = TestTables.create(tableDir, TABLE_NAME, DATE_SCHEMA, PARTITION_BY_DATE, formatVersion); diff --git a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java index 7cb50df919de..3ad236e6c2a1 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java @@ -30,11 +30,9 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; -import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import org.apache.iceberg.exceptions.ValidationException; @@ -201,8 +199,6 @@ private static ByteBuffer longToBuffer(long value) { @BeforeEach public void before() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); this.table = TestTables.create(tableDir, TABLE_NAME, DATE_SCHEMA, PARTITION_SPEC, formatVersion); } diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java index 349912bd78df..15a518d07d7c 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java @@ -22,17 +22,11 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; public class TestPartitionSpecBuilderCaseSensitivity { @@ -57,14 +51,6 @@ public class TestPartitionSpecBuilderCaseSensitivity { required(6, "order_time", Types.TimestampType.withoutZone()), required(7, "ORDER_TIME", Types.TimestampType.withoutZone())); - @TempDir private Path temp; - private File tableDir = null; - - @BeforeEach - public void setupTableDir() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - } - @AfterEach public void cleanupTables() { TestTables.clearTables(); diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java index f5636a77be61..5990c166addb 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java @@ -24,14 +24,10 @@ import static org.assertj.core.api.Assertions.entry; import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; import java.util.Arrays; import java.util.List; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; @@ -39,12 +35,11 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestPartitionSpecInfo { - @TempDir private Path temp; + @TempDir private File tableDir; private final Schema schema = new Schema( required(1, "id", Types.IntegerType.get()), required(2, "data", Types.StringType.get())); - private File tableDir = null; @Parameters(name = "formatVersion = {0}") protected static List parameters() { @@ -53,11 +48,6 @@ protected static List parameters() { @Parameter private int formatVersion; - @BeforeEach - public void setupTableDir() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - } - @AfterEach public void cleanupTables() { TestTables.clearTables(); diff --git a/core/src/test/java/org/apache/iceberg/TestPartitioning.java b/core/src/test/java/org/apache/iceberg/TestPartitioning.java index a4df125f1de2..da04e67bdd66 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitioning.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitioning.java @@ -23,16 +23,12 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -52,13 +48,7 @@ public class TestPartitioning { private static final PartitionSpec BY_DATA_CATEGORY_BUCKET_SPEC = PartitionSpec.builderFor(SCHEMA).identity("data").bucket("category", 8).build(); - @TempDir private Path temp; - private File tableDir = null; - - @BeforeEach - public void setupTableDir() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - } + @TempDir private File tableDir; @AfterEach public void cleanupTables() { diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java index 6fa77ae05c9a..24a9593d6ef6 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java @@ -23,9 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; -import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import org.apache.iceberg.ManifestEntry.Status; @@ -167,9 +165,6 @@ public void testReplaceAndMergeOnePartition() { @TestTemplate public void testReplaceWithUnpartitionedTable() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Table unpartitioned = TestTables.create( tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), formatVersion); @@ -206,9 +201,6 @@ public void testReplaceWithUnpartitionedTable() throws IOException { @TestTemplate public void testReplaceAndMergeWithUnpartitionedTable() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Table unpartitioned = TestTables.create( tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), formatVersion); @@ -369,9 +361,6 @@ public void testValidateWithNullPartition() { @TestTemplate public void testValidateWithVoidTransform() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - Table tableVoid = TestTables.create(tableDir, "tablevoid", SCHEMA, SPEC_VOID, formatVersion); commit(tableVoid, tableVoid.newReplacePartitions().addFile(FILE_A_VOID_PARTITION), branch); diff --git a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java index b1b481dd5305..c4d73b09db59 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java @@ -28,7 +28,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -324,9 +323,6 @@ public void testReplaceTransactionConflict() { @TestTemplate public void testReplaceToCreateAndAppend() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - // this table doesn't exist. Transaction replace = TestTables.beginReplace(tableDir, "test_append", SCHEMA, unpartitioned()); @@ -398,9 +394,6 @@ public void testReplaceTransactionWithUnknownState() { @TestTemplate public void testCreateTransactionWithUnknownState() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - // this table doesn't exist. TestTables.TestTableOperations ops = TestTables.opsWithCommitSucceedButStateUnknown(tableDir, "test_append"); diff --git a/core/src/test/java/org/apache/iceberg/TestSortOrder.java b/core/src/test/java/org/apache/iceberg/TestSortOrder.java index ad773192b417..6e99af3e260e 100644 --- a/core/src/test/java/org/apache/iceberg/TestSortOrder.java +++ b/core/src/test/java/org/apache/iceberg/TestSortOrder.java @@ -28,9 +28,6 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; import java.util.Arrays; import java.util.List; import java.util.Set; @@ -39,7 +36,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SortOrderUtil; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; @@ -70,9 +66,7 @@ public class TestSortOrder { required(30, "ext", Types.StringType.get()), required(42, "Ext1", Types.StringType.get())); - @TempDir private Path temp; - - private File tableDir = null; + @TempDir private File tableDir; @Parameters(name = "formatVersion = {0}") protected static List parameters() { @@ -81,11 +75,6 @@ protected static List parameters() { @Parameter private int formatVersion; - @BeforeEach - public void setupTableDir() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - } - @AfterEach public void cleanupTables() { TestTables.clearTables(); diff --git a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java index ea2142982382..e802a0cf8246 100644 --- a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java +++ b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java @@ -22,9 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.nio.file.Path; import java.util.Arrays; import java.util.List; @@ -64,7 +62,6 @@ protected static List parameters() { @Override @BeforeEach public void setupTable() throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); String tableLocation = tableDir.toURI().toString(); table = TABLES.create(SCHEMA, tableLocation); table diff --git a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java index 656dd782cfe4..78842dbfe8bc 100644 --- a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java @@ -22,9 +22,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; -import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import org.apache.iceberg.types.Types; @@ -56,9 +54,6 @@ public void testPartitionAppend() throws IOException { .withPartitionPath("date=2018-06-08") .build(); - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - this.table = TestTables.create( tableDir, "test_date_partition", dateSchema, partitionSpec, formatVersion); diff --git a/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java b/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java index af39a59971f9..7ffe2d4c391f 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java @@ -80,8 +80,6 @@ private Snapshot appendFileToMain() { @BeforeEach public void before() throws Exception { - tableDir.delete(); // created by table create - this.metadataDir = new File(tableDir, "metadata"); this.table = TestTables.create(tableDir, "test", SCHEMA, SPEC, 2); diff --git a/data/src/test/java/org/apache/iceberg/TestSplitScan.java b/data/src/test/java/org/apache/iceberg/TestSplitScan.java index 85d0571e3104..d8fee2fe360c 100644 --- a/data/src/test/java/org/apache/iceberg/TestSplitScan.java +++ b/data/src/test/java/org/apache/iceberg/TestSplitScan.java @@ -50,7 +50,7 @@ public class TestSplitScan { required(1, "id", Types.IntegerType.get()), required(2, "data", Types.StringType.get())); private Table table; - private File tableLocation; + @TempDir private File tableLocation; private List expectedRecords; @Parameters(name = "fileFormat = {0}") @@ -59,11 +59,9 @@ public static List parameters() { } @Parameter private FileFormat format; - @TempDir private File tempDir; @BeforeEach public void before() throws IOException { - tableLocation = java.nio.file.Files.createTempDirectory(tempDir.toPath(), "table").toFile(); setupTable(); } @@ -100,7 +98,7 @@ private void setupTable() throws IOException { } private File writeToFile(List records, FileFormat fileFormat) throws IOException { - File file = File.createTempFile("junit", null, tempDir); + File file = File.createTempFile("junit", null, tableLocation); assertThat(file.delete()).isTrue(); GenericAppenderFactory factory = diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericReaderDeletes.java b/data/src/test/java/org/apache/iceberg/data/TestGenericReaderDeletes.java index fa44be06ee8f..d7c70919015d 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericReaderDeletes.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericReaderDeletes.java @@ -18,11 +18,8 @@ */ package org.apache.iceberg.data; -import static org.assertj.core.api.Assertions.assertThat; - import java.io.File; import java.io.IOException; -import java.nio.file.Files; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -32,15 +29,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.util.StructLikeSet; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; @ExtendWith(ParameterizedTestExtension.class) public class TestGenericReaderDeletes extends DeleteReadTests { + @TempDir private File tableDir; @Override protected Table createTable(String name, Schema schema, PartitionSpec spec) throws IOException { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); - return TestTables.create(tableDir, name, schema, spec, 2); } diff --git a/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java b/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java index 0b9d60bcc192..f3f25ff317f6 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java @@ -22,7 +22,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Set; @@ -84,9 +83,6 @@ protected static List parameters() { @Override @BeforeEach public void setupTable() throws Exception { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created by table create - this.metadataDir = new File(tableDir, "metadata"); if (partitioned) { diff --git a/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java b/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java index fb93a79d32a3..2ffcf8d15eb5 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java @@ -69,9 +69,6 @@ protected static List parameters() { @Override @BeforeEach public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created by table create - this.metadataDir = new File(tableDir, "metadata"); this.table = create(SCHEMA, PartitionSpec.unpartitioned()); diff --git a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java index 996a434f97f3..a0980b650b3b 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java @@ -26,7 +26,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -95,9 +94,6 @@ protected FileFormat format() { @Override @BeforeEach public void setupTable() throws Exception { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created during table creation - this.metadataDir = new File(tableDir, "metadata"); if (partitioned) { diff --git a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java index 1c8453bd6a75..9813fb0caba2 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java @@ -24,7 +24,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -80,9 +79,6 @@ protected FileFormat format() { @Override @BeforeEach public void setupTable() throws Exception { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created during table creation - this.metadataDir = new File(tableDir, "metadata"); this.table = create(SCHEMA, PartitionSpec.unpartitioned()); this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); diff --git a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java index 177982a59cb3..1e5863e8bff9 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java @@ -22,7 +22,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import org.apache.iceberg.DataFile; @@ -67,9 +66,6 @@ protected FileFormat format() { @Override @BeforeEach public void setupTable() throws Exception { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created during table creation - this.metadataDir = new File(tableDir, "metadata"); this.table = create(SCHEMA, PartitionSpec.unpartitioned()); this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); diff --git a/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java b/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java index 6407fd0cbf70..24798489e781 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java @@ -22,7 +22,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import org.apache.iceberg.FileFormat; @@ -74,9 +73,6 @@ protected FileFormat format() { @Override @BeforeEach public void setupTable() throws Exception { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created during table creation - this.metadataDir = new File(tableDir, "metadata"); if (partitioned) { diff --git a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java index aed549bbf6ca..a1c72ae3b1e0 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java @@ -85,9 +85,6 @@ protected static List parameters() { @Override @BeforeEach public void setupTable() throws IOException { - this.tableDir = java.nio.file.Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created by table create - this.metadataDir = new File(tableDir, "metadata"); this.table = create(SCHEMA, PartitionSpec.unpartitioned()); diff --git a/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java b/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java index e4f6c028bc8e..7617f7fc4920 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java +++ b/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.CharBuffer; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -79,7 +78,7 @@ public abstract class TestWriterMetrics { protected static final Map PROPERTIES = ImmutableMap.of(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none"); - @TempDir private File tempDir; + @TempDir private File tableDir; protected FileFormat fileFormat; protected TestTables.TestTable table = null; @@ -102,9 +101,6 @@ public TestWriterMetrics(FileFormat fileFormat) { @BeforeEach public void setupTable() throws Exception { - File tableDir = Files.createTempDirectory(tempDir.toPath(), "junit").toFile(); - tableDir.delete(); // created by table create - this.table = TestTables.create( tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), SORT_ORDER, FORMAT_V2); @@ -243,9 +239,6 @@ public void testPositionDeleteMetricsCoveringMultipleDataFiles() throws IOExcept @TestTemplate public void testMaxColumns() throws IOException { - File tableDir = Files.createTempDirectory(tempDir.toPath(), "table").toFile(); - assertThat(tableDir.delete()).isTrue(); - int numColumns = TableProperties.METRICS_MAX_INFERRED_COLUMN_DEFAULTS_DEFAULT + 1; List fields = Lists.newArrayListWithCapacity(numColumns); for (int i = 0; i < numColumns; i++) { @@ -306,9 +299,6 @@ public void testMaxColumns() throws IOException { @TestTemplate public void testMaxColumnsWithDefaultOverride() throws IOException { - File tableDir = Files.createTempDirectory(tempDir.toPath(), "table").toFile(); - assertThat(tableDir.delete()).isTrue(); - int numColumns = TableProperties.METRICS_MAX_INFERRED_COLUMN_DEFAULTS_DEFAULT + 1; List fields = Lists.newArrayListWithCapacity(numColumns); for (int i = 0; i < numColumns; i++) { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 21f3ee2c655a..7df167ec3227 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -88,9 +88,6 @@ protected static List parameters() { @Override @BeforeEach public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created by table create - this.metadataDir = new File(tableDir, "metadata"); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index ac5babe11943..75060c479e11 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -107,10 +107,7 @@ protected static List parameters() { @BeforeEach public void setupTable() throws IOException { flinkManifestFolder = Files.createTempDirectory(temp, "flink").toFile(); - - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 9c4f476b02b4..1080362af278 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -23,7 +23,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.time.Duration; import java.util.Arrays; import java.util.List; @@ -75,9 +74,7 @@ protected static List parameters() { @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index 1606ee9f9648..59c618f7a888 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -22,7 +22,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -72,9 +71,7 @@ protected static List parameters() { @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 21f3ee2c655a..7df167ec3227 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -88,9 +88,6 @@ protected static List parameters() { @Override @BeforeEach public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created by table create - this.metadataDir = new File(tableDir, "metadata"); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 7808771d9887..47494cb9bbae 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -107,10 +107,7 @@ protected static List parameters() { @BeforeEach public void setupTable() throws IOException { flinkManifestFolder = Files.createTempDirectory(temp, "flink").toFile(); - - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 9c4f476b02b4..1080362af278 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -23,7 +23,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.time.Duration; import java.util.Arrays; import java.util.List; @@ -75,9 +74,7 @@ protected static List parameters() { @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index 1606ee9f9648..59c618f7a888 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -22,7 +22,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -72,9 +71,7 @@ protected static List parameters() { @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 21f3ee2c655a..7df167ec3227 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -88,9 +88,6 @@ protected static List parameters() { @Override @BeforeEach public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - assertThat(tableDir.delete()).isTrue(); // created by table create - this.metadataDir = new File(tableDir, "metadata"); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 7808771d9887..47494cb9bbae 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -107,10 +107,7 @@ protected static List parameters() { @BeforeEach public void setupTable() throws IOException { flinkManifestFolder = Files.createTempDirectory(temp, "flink").toFile(); - - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 9c4f476b02b4..1080362af278 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -23,7 +23,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.time.Duration; import java.util.Arrays; import java.util.List; @@ -75,9 +74,7 @@ protected static List parameters() { @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index 1606ee9f9648..59c618f7a888 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -22,7 +22,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -72,9 +71,7 @@ protected static List parameters() { @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java index bc094292afa5..4fdbc862ee8c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java @@ -62,12 +62,12 @@ public class TestScanTaskSerialization extends TestBase { optional(3, "c3", Types.StringType.get())); @TempDir private Path temp; + @TempDir private File tableDir; private String tableLocation = null; @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.tableLocation = tableDir.toURI().toString(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index e2ebf7e8817c..6954903b4102 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -77,6 +77,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.io.TempDir; import scala.Option; import scala.Some; import scala.collection.JavaConverters; @@ -142,7 +143,7 @@ public static Object[][] parameters() { } private final String baseTableName = "baseTable"; - private File tableDir; + @TempDir private File tableDir; private String tableLocation; @Parameter(index = 3) @@ -154,11 +155,6 @@ public static Object[][] parameters() { @Override public void before() { super.before(); - try { - this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - } catch (IOException e) { - throw new RuntimeException(e); - } this.tableLocation = tableDir.toURI().toString(); this.catalog = (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 5126b67fc80d..bfa09552396a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -23,7 +23,6 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; -import java.nio.file.Path; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; @@ -112,13 +111,12 @@ public class TestDeleteReachableFilesAction extends TestBase { .withRecordCount(1) .build(); - @TempDir private Path temp; + @TempDir private File tableDir; private Table table; @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = temp.resolve("junit").toFile(); String tableLocation = tableDir.toURI().toString(); this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); spark.conf().set("spark.sql.shuffle.partitions", SHUFFLE_PARTITIONS); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 74169b9f27f1..5909dec51c05 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -121,13 +121,12 @@ public class TestExpireSnapshotsAction extends TestBase { @TempDir private Path temp; - private File tableDir; + @TempDir private File tableDir; private String tableLocation; private Table table; @BeforeEach public void setupTableLocation() throws Exception { - this.tableDir = temp.resolve("junit").toFile(); this.tableLocation = tableDir.toURI().toString(); this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); spark.conf().set("spark.sql.shuffle.partitions", SHUFFLE_PARTITIONS); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java index e15b2fb2174a..3b4dce73fee5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -22,7 +22,6 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.File; -import java.nio.file.Path; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -202,14 +201,13 @@ public class TestRemoveDanglingDeleteAction extends TestBase { .withRecordCount(1) .build(); - @TempDir private Path temp; + @TempDir private File tableDir; private String tableLocation = null; private Table table; @BeforeEach public void before() throws Exception { - File tableDir = temp.resolve("junit").toFile(); this.tableLocation = tableDir.toURI().toString(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 6d9f4de6fb57..12defafff06d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -95,13 +95,11 @@ public abstract class TestRemoveOrphanFilesAction extends TestBase { protected static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).truncate("c2", 2).identity("c3").build(); - @TempDir private java.nio.file.Path temp; - private File tableDir = null; + @TempDir private File tableDir = null; protected String tableLocation = null; @BeforeEach public void setupTableLocation() throws Exception { - this.tableDir = temp.resolve("junit").toFile(); this.tableLocation = tableDir.toURI().toString(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 2de83f8b355c..b7ab47f865b5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -126,6 +126,7 @@ public class TestRewriteDataFilesAction extends TestBase { + @TempDir private File tableDir; private static final int SCALE = 400000; private static final HadoopTables TABLES = new HadoopTables(new Configuration()); @@ -151,7 +152,6 @@ public static void setupSpark() { @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = temp.resolve("junit").toFile(); this.tableLocation = tableDir.toURI().toString(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index b86d74415ac8..79e48f47f241 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -125,10 +125,10 @@ public static Object[] parameters() { private String tableLocation = null; @TempDir private Path temp; + @TempDir private File tableDir; @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = temp.resolve("junit").toFile(); this.tableLocation = tableDir.toURI().toString(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java index ff0b76ed0e3f..35d6e119e86f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java @@ -27,19 +27,16 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopTables; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceHadoopTables extends TestIcebergSourceTablesBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); - - File tableDir = null; + @TempDir private File tableDir; String tableLocation = null; @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.toFile(); - tableDir.delete(); // created by table create - this.tableLocation = tableDir.toURI().toString(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index f0a1a28509e1..c9604863b9fa 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -25,7 +25,6 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -133,12 +132,11 @@ public static void stopSpark() { currentSpark.stop(); } - @TempDir private Path temp; + @TempDir private File tableDir; private String tableLocation = null; @BeforeEach public void setupTableLocation() throws Exception { - File tableDir = temp.toFile(); this.tableLocation = tableDir.toURI().toString(); } From 5bd314bdf6c3b5e0e5346d0f7408353bdf31bc81 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 5 Nov 2024 15:52:24 +0100 Subject: [PATCH 104/313] Core: Support DVs in DeleteFileIndex (#11467) --- .../org/apache/iceberg/DeleteFileIndex.java | 62 ++++++++++++++++--- .../apache/iceberg/util/ContentFileUtil.java | 14 +++++ .../iceberg/DeleteFileIndexTestBase.java | 52 ++++++++++++++++ 3 files changed, 121 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index 8444b91eecd4..ab7fec6fb151 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.ManifestEvaluator; @@ -70,6 +71,7 @@ class DeleteFileIndex { private final PartitionMap eqDeletesByPartition; private final PartitionMap posDeletesByPartition; private final Map posDeletesByPath; + private final Map dvByPath; private final boolean hasEqDeletes; private final boolean hasPosDeletes; private final boolean isEmpty; @@ -78,13 +80,16 @@ private DeleteFileIndex( EqualityDeletes globalDeletes, PartitionMap eqDeletesByPartition, PartitionMap posDeletesByPartition, - Map posDeletesByPath) { + Map posDeletesByPath, + Map dvByPath) { this.globalDeletes = globalDeletes; this.eqDeletesByPartition = eqDeletesByPartition; this.posDeletesByPartition = posDeletesByPartition; this.posDeletesByPath = posDeletesByPath; + this.dvByPath = dvByPath; this.hasEqDeletes = globalDeletes != null || eqDeletesByPartition != null; - this.hasPosDeletes = posDeletesByPartition != null || posDeletesByPath != null; + this.hasPosDeletes = + posDeletesByPartition != null || posDeletesByPath != null || dvByPath != null; this.isEmpty = !hasEqDeletes && !hasPosDeletes; } @@ -125,6 +130,10 @@ public Iterable referencedDeleteFiles() { } } + if (dvByPath != null) { + deleteFiles = Iterables.concat(deleteFiles, dvByPath.values()); + } + return deleteFiles; } @@ -143,9 +152,16 @@ DeleteFile[] forDataFile(long sequenceNumber, DataFile file) { DeleteFile[] global = findGlobalDeletes(sequenceNumber, file); DeleteFile[] eqPartition = findEqPartitionDeletes(sequenceNumber, file); - DeleteFile[] posPartition = findPosPartitionDeletes(sequenceNumber, file); - DeleteFile[] posPath = findPathDeletes(sequenceNumber, file); - return concat(global, eqPartition, posPartition, posPath); + DeleteFile dv = findDV(sequenceNumber, file); + if (dv != null && global == null && eqPartition == null) { + return new DeleteFile[] {dv}; + } else if (dv != null) { + return concat(global, eqPartition, new DeleteFile[] {dv}); + } else { + DeleteFile[] posPartition = findPosPartitionDeletes(sequenceNumber, file); + DeleteFile[] posPath = findPathDeletes(sequenceNumber, file); + return concat(global, eqPartition, posPartition, posPath); + } } private DeleteFile[] findGlobalDeletes(long seq, DataFile dataFile) { @@ -180,6 +196,22 @@ private DeleteFile[] findPathDeletes(long seq, DataFile dataFile) { return deletes == null ? EMPTY_DELETES : deletes.filter(seq); } + private DeleteFile findDV(long seq, DataFile dataFile) { + if (dvByPath == null) { + return null; + } + + DeleteFile dv = dvByPath.get(dataFile.location()); + if (dv != null) { + ValidationException.check( + dv.dataSequenceNumber() >= seq, + "DV data sequence number (%s) must be greater than or equal to data file sequence number (%s)", + dv.dataSequenceNumber(), + seq); + } + return dv; + } + @SuppressWarnings("checkstyle:CyclomaticComplexity") private static boolean canContainEqDeletesForFile( DataFile dataFile, EqualityDeleteFile deleteFile) { @@ -434,11 +466,16 @@ DeleteFileIndex build() { PartitionMap eqDeletesByPartition = PartitionMap.create(specsById); PartitionMap posDeletesByPartition = PartitionMap.create(specsById); Map posDeletesByPath = Maps.newHashMap(); + Map dvByPath = Maps.newHashMap(); for (DeleteFile file : files) { switch (file.content()) { case POSITION_DELETES: - add(posDeletesByPath, posDeletesByPartition, file); + if (ContentFileUtil.isDV(file)) { + add(dvByPath, file); + } else { + add(posDeletesByPath, posDeletesByPartition, file); + } break; case EQUALITY_DELETES: add(globalDeletes, eqDeletesByPartition, file); @@ -453,7 +490,18 @@ DeleteFileIndex build() { globalDeletes.isEmpty() ? null : globalDeletes, eqDeletesByPartition.isEmpty() ? null : eqDeletesByPartition, posDeletesByPartition.isEmpty() ? null : posDeletesByPartition, - posDeletesByPath.isEmpty() ? null : posDeletesByPath); + posDeletesByPath.isEmpty() ? null : posDeletesByPath, + dvByPath.isEmpty() ? null : dvByPath); + } + + private void add(Map dvByPath, DeleteFile dv) { + String path = dv.referencedDataFile(); + DeleteFile existingDV = dvByPath.putIfAbsent(path, dv); + if (existingDV != null) { + throw new ValidationException( + "Can't index multiple DVs for %s: %s and %s", + path, ContentFileUtil.dvDesc(dv), ContentFileUtil.dvDesc(existingDV)); + } } private void add( diff --git a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java index c82b3ff828cf..e4666bd1bd8f 100644 --- a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java @@ -24,6 +24,7 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; @@ -84,4 +85,17 @@ public static String referencedDataFileLocation(DeleteFile deleteFile) { CharSequence location = referencedDataFile(deleteFile); return location != null ? location.toString() : null; } + + public static boolean isDV(DeleteFile deleteFile) { + return deleteFile.format() == FileFormat.PUFFIN; + } + + public static String dvDesc(DeleteFile deleteFile) { + return String.format( + "DV{location=%s, offset=%s, length=%s, referencedDataFile=%s}", + deleteFile.location(), + deleteFile.contentOffset(), + deleteFile.contentSizeInBytes(), + deleteFile.referencedDataFile()); + } } diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java index de7e59ac170d..6ef28191e78e 100644 --- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java @@ -22,20 +22,24 @@ import static org.apache.iceberg.expressions.Expressions.equal; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.UUID; import org.apache.iceberg.DeleteFileIndex.EqualityDeletes; import org.apache.iceberg.DeleteFileIndex.PositionDeletes; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.ContentFileUtil; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -621,4 +625,52 @@ public void testEqualityDeletesGroup() { // it should not be possible to add more elements upon indexing assertThatThrownBy(() -> group.add(SPEC, file1)).isInstanceOf(IllegalStateException.class); } + + @TestTemplate + public void testMixDeleteFilesAndDVs() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + + List deletes = + Arrays.asList( + withDataSequenceNumber(1, partitionedPosDeletes(SPEC, FILE_A.partition())), + withDataSequenceNumber(2, newDV(FILE_A)), + withDataSequenceNumber(1, partitionedPosDeletes(SPEC, FILE_B.partition())), + withDataSequenceNumber(2, partitionedPosDeletes(SPEC, FILE_B.partition()))); + + DeleteFileIndex index = DeleteFileIndex.builderFor(deletes).specsById(table.specs()).build(); + + DeleteFile[] fileADeletes = index.forDataFile(0, FILE_A); + assertThat(fileADeletes).as("Only DV should apply to FILE_A").hasSize(1); + assertThat(ContentFileUtil.isDV(fileADeletes[0])).isTrue(); + assertThat(fileADeletes[0].referencedDataFile()).isEqualTo(FILE_A.location()); + + DeleteFile[] fileBDeletes = index.forDataFile(0, FILE_B); + assertThat(fileBDeletes).as("Two delete files should apply to FILE_B").hasSize(2); + assertThat(ContentFileUtil.isDV(fileBDeletes[0])).isFalse(); + assertThat(ContentFileUtil.isDV(fileBDeletes[1])).isFalse(); + } + + @TestTemplate + public void testMultipleDVs() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + + DeleteFile dv1 = withDataSequenceNumber(1, newDV(FILE_A)); + DeleteFile dv2 = withDataSequenceNumber(2, newDV(FILE_A)); + List dvs = Arrays.asList(dv1, dv2); + + assertThatThrownBy(() -> DeleteFileIndex.builderFor(dvs).specsById(table.specs()).build()) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Can't index multiple DVs for %s", FILE_A.location()); + } + + @TestTemplate + public void testInvalidDVSequenceNumber() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + DeleteFile dv = withDataSequenceNumber(1, newDV(FILE_A)); + List dvs = Collections.singletonList(dv); + DeleteFileIndex index = DeleteFileIndex.builderFor(dvs).specsById(table.specs()).build(); + assertThatThrownBy(() -> index.forDataFile(2, FILE_A)) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("must be greater than or equal to data file sequence number"); + } } From 549674b3fc0cdb18d6cad3e2d6320236fba8c562 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 5 Nov 2024 17:02:00 +0100 Subject: [PATCH 105/313] Core: Adapt commit, scan, and snapshot stats for DVs (#11464) --- .../org/apache/iceberg/SnapshotSummary.java | 25 ++++++- .../iceberg/metrics/CommitMetricsResult.java | 16 +++++ .../metrics/CommitMetricsResultParser.java | 12 ++++ .../apache/iceberg/metrics/ScanMetrics.java | 6 ++ .../iceberg/metrics/ScanMetricsResult.java | 7 ++ .../metrics/ScanMetricsResultParser.java | 6 ++ .../iceberg/metrics/ScanMetricsUtil.java | 7 +- .../apache/iceberg/TestSnapshotSummary.java | 72 +++++++++++++++++++ .../TestCommitMetricsResultParser.java | 12 ++++ .../metrics/TestScanMetricsResultParser.java | 7 ++ .../iceberg/metrics/TestScanReportParser.java | 6 ++ 11 files changed, 173 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java index ad832a5e78e2..6043424cd7fc 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java @@ -25,6 +25,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; 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.apache.iceberg.util.ScanTaskUtil; public class SnapshotSummary { @@ -36,6 +37,8 @@ public class SnapshotSummary { public static final String REMOVED_EQ_DELETE_FILES_PROP = "removed-equality-delete-files"; public static final String ADD_POS_DELETE_FILES_PROP = "added-position-delete-files"; public static final String REMOVED_POS_DELETE_FILES_PROP = "removed-position-delete-files"; + public static final String ADDED_DVS_PROP = "added-dvs"; + public static final String REMOVED_DVS_PROP = "removed-dvs"; public static final String REMOVED_DELETE_FILES_PROP = "removed-delete-files"; public static final String TOTAL_DELETE_FILES_PROP = "total-delete-files"; public static final String ADDED_RECORDS_PROP = "added-records"; @@ -222,6 +225,8 @@ private static class UpdateMetrics { private int removedEqDeleteFiles = 0; private int addedPosDeleteFiles = 0; private int removedPosDeleteFiles = 0; + private int addedDVs = 0; + private int removedDVs = 0; private int addedDeleteFiles = 0; private int removedDeleteFiles = 0; private long addedRecords = 0L; @@ -243,6 +248,8 @@ void clear() { this.removedPosDeleteFiles = 0; this.addedDeleteFiles = 0; this.removedDeleteFiles = 0; + this.addedDVs = 0; + this.removedDVs = 0; this.addedRecords = 0L; this.deletedRecords = 0L; this.addedPosDeletes = 0L; @@ -262,6 +269,8 @@ void addTo(ImmutableMap.Builder builder) { removedPosDeleteFiles > 0, builder, REMOVED_POS_DELETE_FILES_PROP, removedPosDeleteFiles); setIf(addedDeleteFiles > 0, builder, ADDED_DELETE_FILES_PROP, addedDeleteFiles); setIf(removedDeleteFiles > 0, builder, REMOVED_DELETE_FILES_PROP, removedDeleteFiles); + setIf(addedDVs > 0, builder, ADDED_DVS_PROP, addedDVs); + setIf(removedDVs > 0, builder, REMOVED_DVS_PROP, removedDVs); setIf(addedRecords > 0, builder, ADDED_RECORDS_PROP, addedRecords); setIf(deletedRecords > 0, builder, DELETED_RECORDS_PROP, deletedRecords); @@ -283,8 +292,13 @@ void addedFile(ContentFile file) { this.addedRecords += file.recordCount(); break; case POSITION_DELETES: + DeleteFile deleteFile = (DeleteFile) file; + if (ContentFileUtil.isDV(deleteFile)) { + this.addedDVs += 1; + } else { + this.addedPosDeleteFiles += 1; + } this.addedDeleteFiles += 1; - this.addedPosDeleteFiles += 1; this.addedPosDeletes += file.recordCount(); break; case EQUALITY_DELETES: @@ -306,8 +320,13 @@ void removedFile(ContentFile file) { this.deletedRecords += file.recordCount(); break; case POSITION_DELETES: + DeleteFile deleteFile = (DeleteFile) file; + if (ContentFileUtil.isDV(deleteFile)) { + this.removedDVs += 1; + } else { + this.removedPosDeleteFiles += 1; + } this.removedDeleteFiles += 1; - this.removedPosDeleteFiles += 1; this.removedPosDeletes += file.recordCount(); break; case EQUALITY_DELETES: @@ -344,6 +363,8 @@ void merge(UpdateMetrics other) { this.removedEqDeleteFiles += other.removedEqDeleteFiles; this.addedPosDeleteFiles += other.addedPosDeleteFiles; this.removedPosDeleteFiles += other.removedPosDeleteFiles; + this.addedDVs += other.addedDVs; + this.removedDVs += other.removedDVs; this.addedDeleteFiles += other.addedDeleteFiles; this.removedDeleteFiles += other.removedDeleteFiles; this.addedSize += other.addedSize; diff --git a/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResult.java b/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResult.java index ad66e8d32408..7a87172708f6 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResult.java +++ b/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResult.java @@ -34,7 +34,9 @@ public interface CommitMetricsResult { String ADDED_DELETE_FILES = "added-delete-files"; String ADDED_EQ_DELETE_FILES = "added-equality-delete-files"; String ADDED_POS_DELETE_FILES = "added-positional-delete-files"; + String ADDED_DVS = "added-dvs"; String REMOVED_POS_DELETE_FILES = "removed-positional-delete-files"; + String REMOVED_DVS = "removed-dvs"; String REMOVED_EQ_DELETE_FILES = "removed-equality-delete-files"; String REMOVED_DELETE_FILES = "removed-delete-files"; String TOTAL_DELETE_FILES = "total-delete-files"; @@ -75,6 +77,12 @@ public interface CommitMetricsResult { @Nullable CounterResult addedPositionalDeleteFiles(); + @Nullable + @Value.Default + default CounterResult addedDVs() { + return null; + } + @Nullable CounterResult removedDeleteFiles(); @@ -84,6 +92,12 @@ public interface CommitMetricsResult { @Nullable CounterResult removedPositionalDeleteFiles(); + @Nullable + @Value.Default + default CounterResult removedDVs() { + return null; + } + @Nullable CounterResult totalDeleteFiles(); @@ -136,6 +150,7 @@ static CommitMetricsResult from( .addedDeleteFiles(counterFrom(snapshotSummary, SnapshotSummary.ADDED_DELETE_FILES_PROP)) .addedPositionalDeleteFiles( counterFrom(snapshotSummary, SnapshotSummary.ADD_POS_DELETE_FILES_PROP)) + .addedDVs(counterFrom(snapshotSummary, SnapshotSummary.ADDED_DVS_PROP)) .addedEqualityDeleteFiles( counterFrom(snapshotSummary, SnapshotSummary.ADD_EQ_DELETE_FILES_PROP)) .removedDeleteFiles(counterFrom(snapshotSummary, SnapshotSummary.REMOVED_DELETE_FILES_PROP)) @@ -143,6 +158,7 @@ static CommitMetricsResult from( counterFrom(snapshotSummary, SnapshotSummary.REMOVED_EQ_DELETE_FILES_PROP)) .removedPositionalDeleteFiles( counterFrom(snapshotSummary, SnapshotSummary.REMOVED_POS_DELETE_FILES_PROP)) + .removedDVs(counterFrom(snapshotSummary, SnapshotSummary.REMOVED_DVS_PROP)) .totalDeleteFiles(counterFrom(snapshotSummary, SnapshotSummary.TOTAL_DELETE_FILES_PROP)) .addedRecords(counterFrom(snapshotSummary, SnapshotSummary.ADDED_RECORDS_PROP)) .removedRecords(counterFrom(snapshotSummary, SnapshotSummary.DELETED_RECORDS_PROP)) diff --git a/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResultParser.java b/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResultParser.java index d4fd883c4375..2c45581ba5d6 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResultParser.java +++ b/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResultParser.java @@ -81,6 +81,11 @@ static void toJson(CommitMetricsResult metrics, JsonGenerator gen) throws IOExce CounterResultParser.toJson(metrics.addedPositionalDeleteFiles(), gen); } + if (null != metrics.addedDVs()) { + gen.writeFieldName(CommitMetricsResult.ADDED_DVS); + CounterResultParser.toJson(metrics.addedDVs(), gen); + } + if (null != metrics.removedDeleteFiles()) { gen.writeFieldName(CommitMetricsResult.REMOVED_DELETE_FILES); CounterResultParser.toJson(metrics.removedDeleteFiles(), gen); @@ -91,6 +96,11 @@ static void toJson(CommitMetricsResult metrics, JsonGenerator gen) throws IOExce CounterResultParser.toJson(metrics.removedPositionalDeleteFiles(), gen); } + if (null != metrics.removedDVs()) { + gen.writeFieldName(CommitMetricsResult.REMOVED_DVS); + CounterResultParser.toJson(metrics.removedDVs(), gen); + } + if (null != metrics.removedEqualityDeleteFiles()) { gen.writeFieldName(CommitMetricsResult.REMOVED_EQ_DELETE_FILES); CounterResultParser.toJson(metrics.removedEqualityDeleteFiles(), gen); @@ -186,10 +196,12 @@ static CommitMetricsResult fromJson(JsonNode json) { CounterResultParser.fromJson(CommitMetricsResult.ADDED_EQ_DELETE_FILES, json)) .addedPositionalDeleteFiles( CounterResultParser.fromJson(CommitMetricsResult.ADDED_POS_DELETE_FILES, json)) + .addedDVs(CounterResultParser.fromJson(CommitMetricsResult.ADDED_DVS, json)) .removedEqualityDeleteFiles( CounterResultParser.fromJson(CommitMetricsResult.REMOVED_EQ_DELETE_FILES, json)) .removedPositionalDeleteFiles( CounterResultParser.fromJson(CommitMetricsResult.REMOVED_POS_DELETE_FILES, json)) + .removedDVs(CounterResultParser.fromJson(CommitMetricsResult.REMOVED_DVS, json)) .removedDeleteFiles( CounterResultParser.fromJson(CommitMetricsResult.REMOVED_DELETE_FILES, json)) .totalDeleteFiles( diff --git a/core/src/main/java/org/apache/iceberg/metrics/ScanMetrics.java b/core/src/main/java/org/apache/iceberg/metrics/ScanMetrics.java index 421466f0fa85..0f7def37638e 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/ScanMetrics.java +++ b/core/src/main/java/org/apache/iceberg/metrics/ScanMetrics.java @@ -40,6 +40,7 @@ public abstract class ScanMetrics { public static final String INDEXED_DELETE_FILES = "indexed-delete-files"; public static final String EQUALITY_DELETE_FILES = "equality-delete-files"; public static final String POSITIONAL_DELETE_FILES = "positional-delete-files"; + public static final String DVS = "dvs"; public static ScanMetrics noop() { return ScanMetrics.of(MetricsContext.nullMetrics()); @@ -127,6 +128,11 @@ public Counter positionalDeleteFiles() { return metricsContext().counter(POSITIONAL_DELETE_FILES); } + @Value.Derived + public Counter dvs() { + return metricsContext().counter(DVS); + } + public static ScanMetrics of(MetricsContext metricsContext) { return ImmutableScanMetrics.builder().metricsContext(metricsContext).build(); } diff --git a/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsResult.java b/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsResult.java index b930dd83adef..2137e52e0a89 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsResult.java +++ b/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsResult.java @@ -73,6 +73,12 @@ public interface ScanMetricsResult { @Nullable CounterResult positionalDeleteFiles(); + @Nullable + @Value.Default + default CounterResult dvs() { + return null; + } + static ScanMetricsResult fromScanMetrics(ScanMetrics scanMetrics) { Preconditions.checkArgument(null != scanMetrics, "Invalid scan metrics: null"); return ImmutableScanMetricsResult.builder() @@ -93,6 +99,7 @@ static ScanMetricsResult fromScanMetrics(ScanMetrics scanMetrics) { .indexedDeleteFiles(CounterResult.fromCounter(scanMetrics.indexedDeleteFiles())) .equalityDeleteFiles(CounterResult.fromCounter(scanMetrics.equalityDeleteFiles())) .positionalDeleteFiles(CounterResult.fromCounter(scanMetrics.positionalDeleteFiles())) + .dvs(CounterResult.fromCounter(scanMetrics.dvs())) .build(); } } diff --git a/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsResultParser.java b/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsResultParser.java index 5cff1ae8e0db..f85c26753211 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsResultParser.java +++ b/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsResultParser.java @@ -121,6 +121,11 @@ static void toJson(ScanMetricsResult metrics, JsonGenerator gen) throws IOExcept CounterResultParser.toJson(metrics.positionalDeleteFiles(), gen); } + if (null != metrics.dvs()) { + gen.writeFieldName(ScanMetrics.DVS); + CounterResultParser.toJson(metrics.dvs(), gen); + } + gen.writeEndObject(); } @@ -159,6 +164,7 @@ static ScanMetricsResult fromJson(JsonNode json) { .equalityDeleteFiles(CounterResultParser.fromJson(ScanMetrics.EQUALITY_DELETE_FILES, json)) .positionalDeleteFiles( CounterResultParser.fromJson(ScanMetrics.POSITIONAL_DELETE_FILES, json)) + .dvs(CounterResultParser.fromJson(ScanMetrics.DVS, json)) .build(); } } diff --git a/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsUtil.java b/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsUtil.java index 1ba891f58474..6e6aa25636bd 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsUtil.java +++ b/core/src/main/java/org/apache/iceberg/metrics/ScanMetricsUtil.java @@ -21,6 +21,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileContent; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.ScanTaskUtil; public class ScanMetricsUtil { @@ -31,7 +32,11 @@ public static void indexedDeleteFile(ScanMetrics metrics, DeleteFile deleteFile) metrics.indexedDeleteFiles().increment(); if (deleteFile.content() == FileContent.POSITION_DELETES) { - metrics.positionalDeleteFiles().increment(); + if (ContentFileUtil.isDV(deleteFile)) { + metrics.dvs().increment(); + } else { + metrics.positionalDeleteFiles().increment(); + } } else if (deleteFile.content() == FileContent.EQUALITY_DELETES) { metrics.equalityDeleteFiles().increment(); } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java index 529e0cc614f6..b0b9d003e35b 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java @@ -358,4 +358,76 @@ public void rewriteWithDeletesAndDuplicates() { .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "20") .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1"); } + + @TestTemplate + public void testFileSizeSummaryWithDVs() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + + DeleteFile dv1 = newDV(FILE_A); + table.newRowDelta().addDeletes(dv1).commit(); + + DeleteFile dv2 = newDV(FILE_B); + table.newRowDelta().addDeletes(dv2).commit(); + + Map summary1 = table.currentSnapshot().summary(); + long addedPosDeletes1 = dv2.recordCount(); + long addedFileSize1 = dv2.contentSizeInBytes(); + long totalPosDeletes1 = dv1.recordCount() + dv2.recordCount(); + long totalFileSize1 = dv1.contentSizeInBytes() + dv2.contentSizeInBytes(); + assertThat(summary1) + .hasSize(12) + .doesNotContainKey(SnapshotSummary.ADD_POS_DELETE_FILES_PROP) + .doesNotContainKey(SnapshotSummary.REMOVED_POS_DELETE_FILES_PROP) + .containsEntry(SnapshotSummary.ADDED_DELETE_FILES_PROP, "1") + .doesNotContainKey(SnapshotSummary.REMOVED_DELETE_FILES_PROP) + .containsEntry(SnapshotSummary.ADDED_DVS_PROP, "1") + .doesNotContainKey(SnapshotSummary.REMOVED_DVS_PROP) + .containsEntry(SnapshotSummary.ADDED_POS_DELETES_PROP, String.valueOf(addedPosDeletes1)) + .doesNotContainKey(SnapshotSummary.REMOVED_POS_DELETES_PROP) + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, String.valueOf(addedFileSize1)) + .doesNotContainKey(SnapshotSummary.REMOVED_FILE_SIZE_PROP) + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "2") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, String.valueOf(totalPosDeletes1)) + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, String.valueOf(totalFileSize1)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "0") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1"); + + DeleteFile dv3 = newDV(FILE_A); + table + .newRowDelta() + .removeDeletes(dv1) + .removeDeletes(dv2) + .addDeletes(dv3) + .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .commit(); + + Map summary2 = table.currentSnapshot().summary(); + long addedPosDeletes2 = dv3.recordCount(); + long removedPosDeletes2 = dv1.recordCount() + dv2.recordCount(); + long addedFileSize2 = dv3.contentSizeInBytes(); + long removedFileSize2 = dv1.contentSizeInBytes() + dv2.contentSizeInBytes(); + long totalPosDeletes2 = dv3.recordCount(); + long totalFileSize2 = dv3.contentSizeInBytes(); + assertThat(summary2) + .hasSize(16) + .doesNotContainKey(SnapshotSummary.ADD_POS_DELETE_FILES_PROP) + .doesNotContainKey(SnapshotSummary.REMOVED_POS_DELETE_FILES_PROP) + .containsEntry(SnapshotSummary.ADDED_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_DELETE_FILES_PROP, "2") + .containsEntry(SnapshotSummary.ADDED_DVS_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_DVS_PROP, "2") + .containsEntry(SnapshotSummary.ADDED_POS_DELETES_PROP, String.valueOf(addedPosDeletes2)) + .containsEntry(SnapshotSummary.REMOVED_POS_DELETES_PROP, String.valueOf(removedPosDeletes2)) + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, String.valueOf(addedFileSize2)) + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, String.valueOf(removedFileSize2)) + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, String.valueOf(totalPosDeletes2)) + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, String.valueOf(totalFileSize2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "0") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "2"); + } } diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java index 5aa2660143a4..1b51066cf15c 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java @@ -74,6 +74,8 @@ public void roundTripSerde() { .put(SnapshotSummary.ADDED_DELETE_FILES_PROP, "4") .put(SnapshotSummary.ADD_EQ_DELETE_FILES_PROP, "5") .put(SnapshotSummary.ADD_POS_DELETE_FILES_PROP, "6") + .put(SnapshotSummary.ADDED_DVS_PROP, "1") + .put(SnapshotSummary.REMOVED_DVS_PROP, "4") .put(SnapshotSummary.REMOVED_POS_DELETE_FILES_PROP, "7") .put(SnapshotSummary.REMOVED_EQ_DELETE_FILES_PROP, "8") .put(SnapshotSummary.REMOVED_DELETE_FILES_PROP, "9") @@ -101,6 +103,8 @@ public void roundTripSerde() { assertThat(result.addedDeleteFiles().value()).isEqualTo(4L); assertThat(result.addedEqualityDeleteFiles().value()).isEqualTo(5L); assertThat(result.addedPositionalDeleteFiles().value()).isEqualTo(6L); + assertThat(result.addedDVs().value()).isEqualTo(1L); + assertThat(result.removedDVs().value()).isEqualTo(4L); assertThat(result.removedPositionalDeleteFiles().value()).isEqualTo(7L); assertThat(result.removedEqualityDeleteFiles().value()).isEqualTo(8L); assertThat(result.removedDeleteFiles().value()).isEqualTo(9L); @@ -153,6 +157,10 @@ public void roundTripSerde() { + " \"unit\" : \"count\",\n" + " \"value\" : 6\n" + " },\n" + + " \"added-dvs\" : {\n" + + " \"unit\" : \"count\",\n" + + " \"value\" : 1\n" + + " },\n" + " \"removed-delete-files\" : {\n" + " \"unit\" : \"count\",\n" + " \"value\" : 9\n" @@ -161,6 +169,10 @@ public void roundTripSerde() { + " \"unit\" : \"count\",\n" + " \"value\" : 7\n" + " },\n" + + " \"removed-dvs\" : {\n" + + " \"unit\" : \"count\",\n" + + " \"value\" : 4\n" + + " },\n" + " \"removed-equality-delete-files\" : {\n" + " \"unit\" : \"count\",\n" + " \"value\" : 8\n" diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java index 44d5803c4a3a..f5cb1e237307 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java @@ -178,6 +178,7 @@ public void extraFields() { scanMetrics.skippedDeleteManifests().increment(3L); scanMetrics.indexedDeleteFiles().increment(10L); scanMetrics.positionalDeleteFiles().increment(6L); + scanMetrics.dvs().increment(); scanMetrics.equalityDeleteFiles().increment(4L); ScanMetricsResult scanMetricsResult = ScanMetricsResult.fromScanMetrics(scanMetrics); @@ -199,6 +200,7 @@ public void extraFields() { + "\"indexed-delete-files\":{\"unit\":\"count\",\"value\":10}," + "\"equality-delete-files\":{\"unit\":\"count\",\"value\":4}," + "\"positional-delete-files\":{\"unit\":\"count\",\"value\":6}," + + "\"dvs\":{\"unit\":\"count\",\"value\":1}," + "\"extra\": \"value\",\"extra2\":23}")) .isEqualTo(scanMetricsResult); } @@ -242,6 +244,7 @@ public void roundTripSerde() { scanMetrics.skippedDeleteManifests().increment(3L); scanMetrics.indexedDeleteFiles().increment(10L); scanMetrics.positionalDeleteFiles().increment(6L); + scanMetrics.dvs().increment(3L); scanMetrics.equalityDeleteFiles().increment(4L); ScanMetricsResult scanMetricsResult = ScanMetricsResult.fromScanMetrics(scanMetrics); @@ -312,6 +315,10 @@ public void roundTripSerde() { + " \"positional-delete-files\" : {\n" + " \"unit\" : \"count\",\n" + " \"value\" : 6\n" + + " },\n" + + " \"dvs\" : {\n" + + " \"unit\" : \"count\",\n" + + " \"value\" : 3\n" + " }\n" + "}"; diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java index 51e21ad9bf01..caac0704cd3f 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java @@ -84,6 +84,7 @@ public void extraFields() { scanMetrics.skippedDeleteManifests().increment(3L); scanMetrics.indexedDeleteFiles().increment(10L); scanMetrics.positionalDeleteFiles().increment(6L); + scanMetrics.dvs().increment(); scanMetrics.equalityDeleteFiles().increment(4L); String tableName = "roundTripTableName"; @@ -118,6 +119,7 @@ public void extraFields() { + "\"indexed-delete-files\":{\"unit\":\"count\",\"value\":10}," + "\"equality-delete-files\":{\"unit\":\"count\",\"value\":4}," + "\"positional-delete-files\":{\"unit\":\"count\",\"value\":6}," + + "\"dvs\":{\"unit\":\"count\",\"value\":1}," + "\"extra-metric\":\"extra-val\"}," + "\"extra\":\"extraVal\"}")) .isEqualTo(scanReport); @@ -279,6 +281,10 @@ public void roundTripSerde() { + " \"positional-delete-files\" : {\n" + " \"unit\" : \"count\",\n" + " \"value\" : 6\n" + + " },\n" + + " \"dvs\" : {\n" + + " \"unit\" : \"count\",\n" + + " \"value\" : 0\n" + " }\n" + " }\n" + "}"; From ad24d4bf85cd43e1f6337dd4b38fbe2b0e3ffbf4 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 5 Nov 2024 11:15:49 -0700 Subject: [PATCH 106/313] Spark: Synchronously merge new position deletes with old deletes (#11273) --- .../SortingPositionOnlyDeleteWriter.java | 6 +- .../apache/iceberg/util/ContentFileUtil.java | 4 + .../extensions/TestMergeOnReadDelete.java | 83 +++++++++++++ .../extensions/TestMergeOnReadUpdate.java | 99 ++++++++++++++-- .../spark/source/SparkBatchQueryScan.java | 21 ++++ .../spark/source/SparkPositionDeltaWrite.java | 111 +++++++++++++++--- 6 files changed, 294 insertions(+), 30 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java b/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java index 818529c02479..ba954577aa74 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java @@ -167,14 +167,10 @@ private DeleteWriteResult writeDeletes(Collection paths) throws IO private void validatePreviousDeletes(PositionDeleteIndex index) { Preconditions.checkArgument( - index.deleteFiles().stream().allMatch(this::isFileScoped), + index.deleteFiles().stream().allMatch(ContentFileUtil::isFileScoped), "Previous deletes must be file-scoped"); } - private boolean isFileScoped(DeleteFile deleteFile) { - return ContentFileUtil.referencedDataFile(deleteFile) != null; - } - private Collection sort(Collection paths) { if (paths.size() <= 1) { return paths; diff --git a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java index e4666bd1bd8f..9e4a65be02ae 100644 --- a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java @@ -86,6 +86,10 @@ public static String referencedDataFileLocation(DeleteFile deleteFile) { return location != null ? location.toString() : null; } + public static boolean isFileScoped(DeleteFile deleteFile) { + return referencedDataFile(deleteFile) != null; + } + public static boolean isDV(DeleteFile deleteFile) { return deleteFile.format() == FileFormat.PUFFIN; } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index 589543f0375a..60941b8d5560 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -39,9 +39,11 @@ 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.SparkSQLProperties; +import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.iceberg.spark.source.SparkTable; import org.apache.iceberg.spark.source.TestSparkCatalog; import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.Identifier; import org.junit.jupiter.api.BeforeEach; @@ -99,6 +101,87 @@ public void testDeletePartitionGranularity() throws NoSuchTableException { checkDeleteFileGranularity(DeleteGranularity.PARTITION); } + @TestTemplate + public void testPositionDeletesAreMaintainedDuringDelete() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id int, data string) USING iceberg PARTITIONED BY (id) TBLPROPERTIES" + + "('%s'='%s', '%s'='%s', '%s'='%s')", + tableName, + TableProperties.FORMAT_VERSION, + 2, + TableProperties.DELETE_MODE, + "merge-on-read", + TableProperties.DELETE_GRANULARITY, + "file"); + createBranchIfNeeded(); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "b"), + new SimpleRecord(1, "c"), + new SimpleRecord(2, "d"), + new SimpleRecord(2, "e")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(commitTarget()) + .append(); + + sql("DELETE FROM %s WHERE id = 1 and data='a'", commitTarget()); + sql("DELETE FROM %s WHERE id = 2 and data='d'", commitTarget()); + sql("DELETE FROM %s WHERE id = 1 and data='c'", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); + assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "b"), row(2, "e")), + sql("SELECT * FROM %s ORDER BY id ASC", selectTarget())); + } + + @TestTemplate + public void testUnpartitionedPositionDeletesAreMaintainedDuringDelete() + throws NoSuchTableException { + sql( + "CREATE TABLE %s (id int, data string) USING iceberg TBLPROPERTIES" + + "('%s'='%s', '%s'='%s', '%s'='%s')", + tableName, + TableProperties.FORMAT_VERSION, + 2, + TableProperties.DELETE_MODE, + "merge-on-read", + TableProperties.DELETE_GRANULARITY, + "file"); + createBranchIfNeeded(); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "b"), + new SimpleRecord(1, "c"), + new SimpleRecord(2, "d"), + new SimpleRecord(2, "e")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(commitTarget()) + .append(); + + sql("DELETE FROM %s WHERE id = 1 and data='a'", commitTarget()); + sql("DELETE FROM %s WHERE id = 2 and data='d'", commitTarget()); + sql("DELETE FROM %s WHERE id = 1 and data='c'", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); + assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "b"), row(2, "e")), + sql("SELECT * FROM %s ORDER BY id ASC", selectTarget())); + } + private void checkDeleteFileGranularity(DeleteGranularity deleteGranularity) throws NoSuchTableException { createAndInitPartitionedTable(); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java index 391fae4ea696..e9cc9d8541ad 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Map; import org.apache.iceberg.ParameterizedTestExtension; @@ -55,19 +56,82 @@ public void testUpdatePartitionGranularity() { checkUpdateFileGranularity(DeleteGranularity.PARTITION); } - private void checkUpdateFileGranularity(DeleteGranularity deleteGranularity) { - createAndInitTable("id INT, dep STRING", "PARTITIONED BY (dep)", null /* empty */); + @TestTemplate + public void testUpdateFileGranularityMergesDeleteFiles() { + // Range distribution will produce partition scoped deletes which will not be cleaned up + assumeThat(distributionMode).isNotEqualToIgnoringCase("range"); - sql( - "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); + checkUpdateFileGranularity(DeleteGranularity.FILE); + sql("UPDATE %s SET id = id + 1 WHERE id = 4", commitTarget()); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + String expectedDeleteFilesCount = "2"; + validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); - append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); - append(tableName, "{ \"id\": 3, \"dep\": \"hr\" }\n" + "{ \"id\": 4, \"dep\": \"hr\" }"); - append(tableName, "{ \"id\": 1, \"dep\": \"it\" }\n" + "{ \"id\": 2, \"dep\": \"it\" }"); - append(tableName, "{ \"id\": 3, \"dep\": \"it\" }\n" + "{ \"id\": 4, \"dep\": \"it\" }"); + assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(0, "hr"), + row(2, "hr"), + row(2, "hr"), + row(5, "hr"), + row(0, "it"), + row(2, "it"), + row(2, "it"), + row(5, "it")), + sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); + } - createBranchIfNeeded(); + @TestTemplate + public void testUpdateUnpartitionedFileGranularityMergesDeleteFiles() { + // Range distribution will produce partition scoped deletes which will not be cleaned up + assumeThat(distributionMode).isNotEqualToIgnoringCase("range"); + initTable("", DeleteGranularity.FILE); + + sql("UPDATE %s SET id = id - 1 WHERE id = 1 OR id = 3", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(5); + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + String expectedDeleteFilesCount = "4"; + validateMergeOnRead(currentSnapshot, "1", expectedDeleteFilesCount, "1"); + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(0, "hr"), + row(2, "hr"), + row(2, "hr"), + row(4, "hr"), + row(0, "it"), + row(2, "it"), + row(2, "it"), + row(4, "it")), + sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); + + sql("UPDATE %s SET id = id + 1 WHERE id = 4", commitTarget()); + table.refresh(); + currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + expectedDeleteFilesCount = "2"; + + validateMergeOnRead(currentSnapshot, "1", expectedDeleteFilesCount, "1"); + assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(0, "hr"), + row(2, "hr"), + row(2, "hr"), + row(5, "hr"), + row(0, "it"), + row(2, "it"), + row(2, "it"), + row(5, "it")), + sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); + } + + private void checkUpdateFileGranularity(DeleteGranularity deleteGranularity) { + initTable("PARTITIONED BY (dep)", deleteGranularity); sql("UPDATE %s SET id = id - 1 WHERE id = 1 OR id = 3", commitTarget()); @@ -91,4 +155,19 @@ private void checkUpdateFileGranularity(DeleteGranularity deleteGranularity) { row(4, "it")), sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); } + + private void initTable(String partitionedBy, DeleteGranularity deleteGranularity) { + createAndInitTable("id INT, dep STRING", partitionedBy, null /* empty */); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); + + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 3, \"dep\": \"hr\" }\n" + "{ \"id\": 4, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 1, \"dep\": \"it\" }\n" + "{ \"id\": 2, \"dep\": \"it\" }"); + append(tableName, "{ \"id\": 3, \"dep\": \"it\" }\n" + "{ \"id\": 4, \"dep\": \"it\" }"); + + createBranchIfNeeded(); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index 66cda5b82955..18e483f23fc6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -24,6 +24,8 @@ import java.util.Set; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionScanTask; import org.apache.iceberg.PartitionSpec; @@ -48,6 +50,8 @@ import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkV2Filters; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.expressions.NamedReference; @@ -158,6 +162,23 @@ public void filter(Predicate[] predicates) { } } + protected Map rewritableDeletes() { + Map rewritableDeletes = Maps.newHashMap(); + + for (ScanTask task : tasks()) { + FileScanTask fileScanTask = task.asFileScanTask(); + for (DeleteFile deleteFile : fileScanTask.deletes()) { + if (ContentFileUtil.isFileScoped(deleteFile)) { + rewritableDeletes + .computeIfAbsent(fileScanTask.file().location(), ignored -> DeleteFileSet.create()) + .add(deleteFile); + } + } + } + + return rewritableDeletes; + } + // at this moment, Spark can only pass IN filters for a single attribute // if there are multiple filter attributes, Spark will pass two separate IN filters private Expression convertRuntimeFilters(Predicate[] predicates) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index 02c87b53e762..18020ee935b6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -28,6 +28,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.function.Function; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -42,8 +43,12 @@ import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteLoader; import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.encryption.EncryptingFileIO; import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -67,6 +72,7 @@ import org.apache.iceberg.spark.SparkWriteRequirements; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.iceberg.util.StructProjection; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; @@ -166,10 +172,23 @@ private class PositionDeltaBatchWrite implements DeltaBatchWrite { @Override public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - // broadcast the table metadata as the writer factory will be sent to executors - Broadcast tableBroadcast = - sparkContext.broadcast(SerializableTableWithSize.copyOf(table)); - return new PositionDeltaWriteFactory(tableBroadcast, command, context, writeProperties); + // broadcast large objects since the writer factory will be sent to executors + return new PositionDeltaWriteFactory( + sparkContext.broadcast(SerializableTableWithSize.copyOf(table)), + broadcastRewritableDeletes(), + command, + context, + writeProperties); + } + + private Broadcast> broadcastRewritableDeletes() { + if (context.deleteGranularity() == DeleteGranularity.FILE && scan != null) { + Map rewritableDeletes = scan.rewritableDeletes(); + if (rewritableDeletes != null && !rewritableDeletes.isEmpty()) { + return sparkContext.broadcast(rewritableDeletes); + } + } + return null; } @Override @@ -185,6 +204,7 @@ public void commit(WriterCommitMessage[] messages) { int addedDataFilesCount = 0; int addedDeleteFilesCount = 0; + int removedDeleteFilesCount = 0; for (WriterCommitMessage message : messages) { DeltaTaskCommit taskCommit = (DeltaTaskCommit) message; @@ -199,6 +219,11 @@ public void commit(WriterCommitMessage[] messages) { addedDeleteFilesCount += 1; } + for (DeleteFile deleteFile : taskCommit.rewrittenDeleteFiles()) { + rowDelta.removeDeletes(deleteFile); + removedDeleteFilesCount += 1; + } + referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles())); } @@ -227,10 +252,11 @@ public void commit(WriterCommitMessage[] messages) { String commitMsg = String.format( - "position delta with %d data files and %d delete files " + "position delta with %d data files, %d delete files and %d rewritten delete files" + "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)", addedDataFilesCount, addedDeleteFilesCount, + removedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel); @@ -314,18 +340,21 @@ private void commitOperation(SnapshotUpdate operation, String description) { public static class DeltaTaskCommit implements WriterCommitMessage { private final DataFile[] dataFiles; private final DeleteFile[] deleteFiles; + private final DeleteFile[] rewrittenDeleteFiles; private final CharSequence[] referencedDataFiles; DeltaTaskCommit(WriteResult result) { this.dataFiles = result.dataFiles(); this.deleteFiles = result.deleteFiles(); this.referencedDataFiles = result.referencedDataFiles(); + this.rewrittenDeleteFiles = result.rewrittenDeleteFiles(); } DeltaTaskCommit(DeleteWriteResult result) { this.dataFiles = new DataFile[0]; this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]); this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]); + this.rewrittenDeleteFiles = result.rewrittenDeleteFiles().toArray(new DeleteFile[0]); } DataFile[] dataFiles() { @@ -336,6 +365,10 @@ DeleteFile[] deleteFiles() { return deleteFiles; } + DeleteFile[] rewrittenDeleteFiles() { + return rewrittenDeleteFiles; + } + CharSequence[] referencedDataFiles() { return referencedDataFiles; } @@ -343,16 +376,19 @@ CharSequence[] referencedDataFiles() { private static class PositionDeltaWriteFactory implements DeltaWriterFactory { private final Broadcast
    tableBroadcast; + private final Broadcast> rewritableDeletesBroadcast; private final Command command; private final Context context; private final Map writeProperties; PositionDeltaWriteFactory( Broadcast
    tableBroadcast, + Broadcast> rewritableDeletesBroadcast, Command command, Context context, Map writeProperties) { this.tableBroadcast = tableBroadcast; + this.rewritableDeletesBroadcast = rewritableDeletesBroadcast; this.command = command; this.context = context; this.writeProperties = writeProperties; @@ -385,17 +421,22 @@ public DeltaWriter createWriter(int partitionId, long taskId) { .build(); if (command == DELETE) { - return new DeleteOnlyDeltaWriter(table, writerFactory, deleteFileFactory, context); + return new DeleteOnlyDeltaWriter( + table, rewritableDeletes(), writerFactory, deleteFileFactory, context); } else if (table.spec().isUnpartitioned()) { return new UnpartitionedDeltaWriter( - table, writerFactory, dataFileFactory, deleteFileFactory, context); + table, rewritableDeletes(), writerFactory, dataFileFactory, deleteFileFactory, context); } else { return new PartitionedDeltaWriter( - table, writerFactory, dataFileFactory, deleteFileFactory, context); + table, rewritableDeletes(), writerFactory, dataFileFactory, deleteFileFactory, context); } } + + private Map rewritableDeletes() { + return rewritableDeletesBroadcast != null ? rewritableDeletesBroadcast.getValue() : null; + } } private abstract static class BaseDeltaWriter implements DeltaWriter { @@ -437,23 +478,58 @@ protected PartitioningWriter newDataWriter( // use a fanout writer if the input is unordered no matter whether fanout writers are enabled // clustered writers assume that the position deletes are already ordered by file and position protected PartitioningWriter, DeleteWriteResult> newDeleteWriter( - Table table, SparkFileWriterFactory writers, OutputFileFactory files, Context context) { + Table table, + Map rewritableDeletes, + SparkFileWriterFactory writers, + OutputFileFactory files, + Context context) { FileIO io = table.io(); boolean inputOrdered = context.inputOrdered(); long targetFileSize = context.targetDeleteFileSize(); DeleteGranularity deleteGranularity = context.deleteGranularity(); - if (inputOrdered) { + if (inputOrdered && rewritableDeletes == null) { return new ClusteredPositionDeleteWriter<>( writers, files, io, targetFileSize, deleteGranularity); } else { return new FanoutPositionOnlyDeleteWriter<>( - writers, files, io, targetFileSize, deleteGranularity); + writers, + files, + io, + targetFileSize, + deleteGranularity, + rewritableDeletes != null + ? new PreviousDeleteLoader(table, rewritableDeletes) + : path -> null /* no previous file scoped deletes */); } } } + private static class PreviousDeleteLoader implements Function { + private final Map deleteFiles; + private final DeleteLoader deleteLoader; + + PreviousDeleteLoader(Table table, Map deleteFiles) { + this.deleteFiles = deleteFiles; + this.deleteLoader = + new BaseDeleteLoader( + deleteFile -> + EncryptingFileIO.combine(table.io(), table.encryption()) + .newInputFile(deleteFile)); + } + + @Override + public PositionDeleteIndex apply(CharSequence path) { + DeleteFileSet deleteFileSet = deleteFiles.get(path.toString()); + if (deleteFileSet == null) { + return null; + } + + return deleteLoader.loadPositionDeletes(deleteFileSet, path); + } + } + private static class DeleteOnlyDeltaWriter extends BaseDeltaWriter { private final PartitioningWriter, DeleteWriteResult> delegate; private final PositionDelete positionDelete; @@ -470,11 +546,13 @@ private static class DeleteOnlyDeltaWriter extends BaseDeltaWriter { DeleteOnlyDeltaWriter( Table table, + Map rewritableDeletes, SparkFileWriterFactory writerFactory, OutputFileFactory deleteFileFactory, Context context) { - this.delegate = newDeleteWriter(table, writerFactory, deleteFileFactory, context); + this.delegate = + newDeleteWriter(table, rewritableDeletes, writerFactory, deleteFileFactory, context); this.positionDelete = PositionDelete.create(); this.io = table.io(); this.specs = table.specs(); @@ -557,6 +635,7 @@ private abstract static class DeleteAndDataDeltaWriter extends BaseDeltaWriter { DeleteAndDataDeltaWriter( Table table, + Map rewritableDeletes, SparkFileWriterFactory writerFactory, OutputFileFactory dataFileFactory, OutputFileFactory deleteFileFactory, @@ -564,7 +643,7 @@ private abstract static class DeleteAndDataDeltaWriter extends BaseDeltaWriter { this.delegate = new BasePositionDeltaWriter<>( newDataWriter(table, writerFactory, dataFileFactory, context), - newDeleteWriter(table, writerFactory, deleteFileFactory, context)); + newDeleteWriter(table, rewritableDeletes, writerFactory, deleteFileFactory, context)); this.io = table.io(); this.specs = table.specs(); @@ -629,11 +708,12 @@ private static class UnpartitionedDeltaWriter extends DeleteAndDataDeltaWriter { UnpartitionedDeltaWriter( Table table, + Map rewritableDeletes, SparkFileWriterFactory writerFactory, OutputFileFactory dataFileFactory, OutputFileFactory deleteFileFactory, Context context) { - super(table, writerFactory, dataFileFactory, deleteFileFactory, context); + super(table, rewritableDeletes, writerFactory, dataFileFactory, deleteFileFactory, context); this.dataSpec = table.spec(); } @@ -655,11 +735,12 @@ private static class PartitionedDeltaWriter extends DeleteAndDataDeltaWriter { PartitionedDeltaWriter( Table table, + Map rewritableDeletes, SparkFileWriterFactory writerFactory, OutputFileFactory dataFileFactory, OutputFileFactory deleteFileFactory, Context context) { - super(table, writerFactory, dataFileFactory, deleteFileFactory, context); + super(table, rewritableDeletes, writerFactory, dataFileFactory, deleteFileFactory, context); this.dataSpec = table.spec(); this.dataPartitionKey = new PartitionKey(dataSpec, context.dataSchema()); From 9be7f00dd6a9fb480a94c46d49473334908be859 Mon Sep 17 00:00:00 2001 From: Marc Cenac <547446+mrcnc@users.noreply.github.com> Date: Tue, 5 Nov 2024 17:55:56 -0600 Subject: [PATCH 107/313] Fix ADLSLocation file parsing (#11395) * Azure: Fix ADLSLocation file parsing * Azure: Remove invalid test cases from ADLSLocationTest * Update Javadocs with reference to ADLS URI --- .../iceberg/azure/adlsv2/ADLSLocation.java | 13 +++++----- .../azure/adlsv2/ADLSLocationTest.java | 24 +++++-------------- 2 files changed, 13 insertions(+), 24 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java index e73093512b82..5af590628fe8 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java @@ -25,14 +25,16 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** - * This class represents a fully qualified location in Azure expressed as a URI. + * This class represents a fully qualified location to a file or directory in Azure Data Lake + * Storage Gen2 storage. * - *

    Locations follow the conventions used by Hadoop's Azure support, i.e. + *

    Locations follow a URI like structure to identify resources * *

    {@code abfs[s]://[@]/}
    * - *

    See Hadoop Azure - * Support + *

    See Azure + * Data Lake Storage URI */ class ADLSLocation { private static final Pattern URI_PATTERN = Pattern.compile("^abfss?://([^/?#]+)(.*)?$"); @@ -64,8 +66,7 @@ class ADLSLocation { } String uriPath = matcher.group(2); - uriPath = uriPath == null ? "" : uriPath.startsWith("/") ? uriPath.substring(1) : uriPath; - this.path = uriPath.split("\\?", -1)[0].split("#", -1)[0]; + this.path = uriPath == null ? "" : uriPath.startsWith("/") ? uriPath.substring(1) : uriPath; } /** Returns Azure storage account. */ diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java index 867b54b4c7e3..403886f4b28e 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java @@ -82,23 +82,11 @@ public void testNoPath() { assertThat(location.path()).isEqualTo(""); } - @Test - public void testQueryAndFragment() { - String p1 = "abfs://container@account.dfs.core.windows.net/path/to/file?query=foo#123"; - ADLSLocation location = new ADLSLocation(p1); - - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); - assertThat(location.container().get()).isEqualTo("container"); - assertThat(location.path()).isEqualTo("path/to/file"); - } - - @Test - public void testQueryAndFragmentNoPath() { - String p1 = "abfs://container@account.dfs.core.windows.net?query=foo#123"; - ADLSLocation location = new ADLSLocation(p1); - - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); - assertThat(location.container().get()).isEqualTo("container"); - assertThat(location.path()).isEqualTo(""); + @ParameterizedTest + @ValueSource(strings = {"file?.txt", "file%3F.txt"}) + public void testQuestionMarkInFileName(String path) { + String fullPath = String.format("abfs://container@account.dfs.core.windows.net/%s", path); + ADLSLocation location = new ADLSLocation(fullPath); + assertThat(location.path()).contains(path); } } From 2ffd3b02a2aa8c02ea0a322c0410b1bffcbc62fa Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Wed, 6 Nov 2024 13:47:35 +0530 Subject: [PATCH 108/313] open-api: Build runtime jar for test fixture (#11279) --- build.gradle | 28 ++- deploy.gradle | 1 + open-api/LICENSE | 555 +++++++++++++++++++++++++++++++++++++++++++++++ open-api/NOTICE | 226 +++++++++++++++++++ 4 files changed, 807 insertions(+), 3 deletions(-) create mode 100644 open-api/LICENSE create mode 100644 open-api/NOTICE diff --git a/build.gradle b/build.gradle index ad4d77f9d161..d5bb33b7ff25 100644 --- a/build.gradle +++ b/build.gradle @@ -947,6 +947,9 @@ project(':iceberg-snowflake') { project(':iceberg-open-api') { apply plugin: 'java-test-fixtures' + apply plugin: 'com.gradleup.shadow' + + build.dependsOn shadowJar dependencies { testImplementation project(':iceberg-api') @@ -970,11 +973,9 @@ project(':iceberg-open-api') { testFixturesImplementation project(':iceberg-gcp') testFixturesImplementation project(':iceberg-azure') testFixturesImplementation(libs.hadoop3.common) { - exclude group: 'log4j' exclude group: 'org.slf4j' exclude group: 'ch.qos.reload4j' exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'com.fasterxml.woodstox' exclude group: 'com.google.guava' exclude group: 'com.google.protobuf' exclude group: 'org.apache.curator' @@ -983,7 +984,6 @@ project(':iceberg-open-api') { exclude group: 'org.apache.hadoop', module: 'hadoop-auth' exclude group: 'org.apache.commons', module: 'commons-configuration2' exclude group: 'org.apache.hadoop.thirdparty', module: 'hadoop-shaded-protobuf_3_7' - exclude group: 'org.codehaus.woodstox' exclude group: 'org.eclipse.jetty' } testFixturesImplementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') @@ -1017,6 +1017,28 @@ project(':iceberg-open-api') { recommend.set(true) } check.dependsOn('validateRESTCatalogSpec') + + shadowJar { + archiveBaseName.set("iceberg-open-api-test-fixtures-runtime") + archiveClassifier.set(null) + configurations = [project.configurations.testFixturesRuntimeClasspath] + from sourceSets.testFixtures.output + zip64 true + + // include the LICENSE and NOTICE files for the runtime Jar + from(projectDir) { + include 'LICENSE' + include 'NOTICE' + } + + manifest { + attributes 'Main-Class': 'org.apache.iceberg.rest.RESTCatalogServer' + } + } + + jar { + enabled = false + } } @Memoized diff --git a/deploy.gradle b/deploy.gradle index 37a6262f1efe..2c7b323bf82f 100644 --- a/deploy.gradle +++ b/deploy.gradle @@ -75,6 +75,7 @@ subprojects { } else if (isOpenApi) { artifact testJar artifact testFixturesJar + artifact shadowJar } else { if (tasks.matching({task -> task.name == 'shadowJar'}).isEmpty()) { from components.java diff --git a/open-api/LICENSE b/open-api/LICENSE new file mode 100644 index 000000000000..ee4389e3ef8e --- /dev/null +++ b/open-api/LICENSE @@ -0,0 +1,555 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. + +-------------------------------------------------------------------------------- + +This product includes a gradle wrapper. + +* gradlew and gradle/wrapper/gradle-wrapper.properties + +Copyright: 2010-2019 Gradle Authors. +Home page: https://github.com/gradle/gradle +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache Avro. + +* Conversion in DecimalWriter is based on Avro's Conversions.DecimalConversion. + +Copyright: 2014-2017 The Apache Software Foundation. +Home page: https://avro.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache Parquet. + +* DynMethods.java +* DynConstructors.java +* IOUtil.java readFully and tests +* ByteBufferInputStream implementations and tests + +Copyright: 2014-2017 The Apache Software Foundation. +Home page: https://parquet.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Cloudera Kite. + +* SchemaVisitor and visit methods + +Copyright: 2013-2017 Cloudera Inc. +Home page: https://kitesdk.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Presto. + +* Retry wait and jitter logic in Tasks.java +* S3FileIO logic derived from PrestoS3FileSystem.java in S3InputStream.java + and S3OutputStream.java +* SQL grammar rules for parsing CALL statements in IcebergSqlExtensions.g4 +* some aspects of handling stored procedures + +Copyright: 2016 Facebook and contributors +Home page: https://prestodb.io/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache iBATIS. + +* Hive ScriptRunner.java + +Copyright: 2004 Clinton Begin +Home page: https://ibatis.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache Hive. + +* Hive metastore derby schema in hive-schema-3.1.0.derby.sql + +Copyright: 2011-2018 The Apache Software Foundation +Home page: https://hive.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache Spark. + +* dev/check-license script +* vectorized reading of definition levels in BaseVectorizedParquetValuesReader.java +* portions of the extensions parser +* casting logic in AssignmentAlignmentSupport +* implementation of SetAccumulator. +* Connector expressions. + +Copyright: 2011-2018 The Apache Software Foundation +Home page: https://spark.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Delta Lake. + +* AssignmentAlignmentSupport is an independent development but UpdateExpressionsSupport in Delta was used as a reference. + +Copyright: 2020 The Delta Lake Project Authors. +Home page: https://delta.io/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache Commons. + +* Core ArrayUtil. + +Copyright: 2020 The Apache Software Foundation +Home page: https://commons.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache HttpComponents Client. + +* retry and error handling logic in ExponentialHttpRequestRetryStrategy.java + +Copyright: 1999-2022 The Apache Software Foundation. +Home page: https://hc.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache Flink. + +* Parameterized test at class level logic in ParameterizedTestExtension.java +* Parameter provider annotation for parameterized tests in Parameters.java +* Parameter field annotation for parameterized tests in Parameter.java + +Copyright: 1999-2022 The Apache Software Foundation. +Home page: https://flink.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains code from the following projects: + +-------------------------------------------------------------------------------- +Group: com.fasterxml.jackson Name: jackson-bom Version: 2.18.0 +Project URL (from POM): https://github.com/FasterXML/jackson-bom +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-annotations Version: 2.18.0 +Project URL: https://github.com/FasterXML/jackson +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-core Version: 2.18.0 +Project URL: https://github.com/FasterXML/jackson-core +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.fasterxml.jackson.core Name: jackson-databind Version: 2.18.0 +Project URL: https://github.com/FasterXML/jackson +License (from POM): Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.github.ben-manes.caffeine Name: caffeine Version: 2.9.3 +Project URL (from POM): https://github.com/ben-manes/caffeine +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- +Group: com.google.errorprone Name: error_prone_annotations Version: 2.10.0 +License (from POM): Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-codec Name: commons-codec Version: 1.17.0 +Project URL: https://commons.apache.org/proper/commons-codec/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: commons-io Name: commons-io Version: 2.16.1 +Project URL: https://commons.apache.org/proper/commons-io/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: dev.failsafe Name: failsafe Version: 3.3.2 +License (from POM): Apache License, Version 2.0 - http://apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +Group: io.airlift Name: aircompressor Version: 0.27 +Project URL (from POM): https://github.com/airlift/aircompressor +License (from POM): Apache License 2.0 - https://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +Group: org.apache.avro Name: avro Version: 1.12.0 +Project URL: https://avro.apache.org +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-compress Version: 1.26.2 +Project URL: https://commons.apache.org/proper/commons-compress/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.commons Name: commons-lang3 Version: 3.14.0 +Project URL: https://commons.apache.org/proper/commons-lang/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.hadoop Name: hadoop-common Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.client5 Name: httpclient5 Version: 5.4 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.apache.httpcomponents.core5 Name: httpcore5 Version: 5.3 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +Group: org.apache.httpcomponents.core5 Name: httpcore5-h2 Version: 5.3 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.checkerframework Name: checker-qual Version: 3.19.0 +Project URL (from POM): https://checkerframework.org +License (from POM): The MIT License - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-http Version: 11.0.24 +Project URL: https://eclipse.dev/jetty/ +License (from POM): Apache Software License - Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 2.0 - https://www.eclipse.org/legal/epl-2.0/ + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-io Version: 11.0.24 +Project URL: https://eclipse.dev/jetty/ +License (from POM): Apache Software License - Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 2.0 - https://www.eclipse.org/legal/epl-2.0/ + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-security Version: 11.0.24 +Project URL: https://eclipse.dev/jetty/ +License (from POM): Apache Software License - Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 2.0 - https://www.eclipse.org/legal/epl-2.0/ + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-server Version: 11.0.24 +Project URL: https://eclipse.dev/jetty/ +License (from POM): Apache Software License - Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 2.0 - https://www.eclipse.org/legal/epl-2.0/ + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-servlet Version: 11.0.24 +Project URL: https://eclipse.dev/jetty/ +License (from POM): Apache Software License - Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 2.0 - https://www.eclipse.org/legal/epl-2.0/ + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty Name: jetty-util Version: 11.0.24 +Project URL: https://eclipse.dev/jetty/ +License (from POM): Apache Software License - Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 2.0 - https://www.eclipse.org/legal/epl-2.0/ + +-------------------------------------------------------------------------------- + +Group: org.eclipse.jetty.toolchain Name: jetty-jakarta-servlet-api Version: 5.0.2 +Project URL (from POM): https://eclipse.org/jetty +License (from POM): Apache Software License - Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 +License (from POM): Eclipse Public License - Version 1.0 - http://www.eclipse.org/org/documents/epl-v10.php + +-------------------------------------------------------------------------------- + +Group: org.junit Name: junit-bom Version: 5.11.3 +Project URL (from POM): https://junit.org/junit5/ +License (from POM): Eclipse Public License v2.0 - https://www.eclipse.org/legal/epl-v20.html + +-------------------------------------------------------------------------------- + +Group: org.junit.jupiter Name: junit-jupiter Version: 5.11.3 +Project URL (from POM): https://junit.org/junit5/ +License (from POM): Eclipse Public License v2.0 - https://www.eclipse.org/legal/epl-v20.html + +-------------------------------------------------------------------------------- + +Group: org.junit.jupiter Name: junit-jupiter-api Version: 5.11.3 +Project URL (from POM): https://junit.org/junit5/ +License (from POM): Eclipse Public License v2.0 - https://www.eclipse.org/legal/epl-v20.html + +-------------------------------------------------------------------------------- + +Group: org.junit.jupiter Name: junit-jupiter-engine Version: 5.11.3 +Project URL (from POM): https://junit.org/junit5/ +License (from POM): Eclipse Public License v2.0 - https://www.eclipse.org/legal/epl-v20.html + +-------------------------------------------------------------------------------- + +Group: org.junit.jupiter Name: junit-jupiter-params Version: 5.11.3 +Project URL (from POM): https://junit.org/junit5/ +License (from POM): Eclipse Public License v2.0 - https://www.eclipse.org/legal/epl-v20.html + +-------------------------------------------------------------------------------- + +Group: org.junit.platform Name: junit-platform-commons Version: 1.11.3 +Project URL (from POM): https://junit.org/junit5/ +License (from POM): Eclipse Public License v2.0 - https://www.eclipse.org/legal/epl-v20.html + +-------------------------------------------------------------------------------- + +Group: org.junit.platform Name: junit-platform-engine Version: 1.11.3 +Project URL (from POM): https://junit.org/junit5/ +License (from POM): Eclipse Public License v2.0 - https://www.eclipse.org/legal/epl-v20.html + +-------------------------------------------------------------------------------- + +Group: org.opentest4j Name: opentest4j Version: 1.3.0 +Project URL (from POM): https://github.com/ota4j-team/opentest4j +License (from POM): The Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.roaringbitmap Name: RoaringBitmap Version: 1.3.0 +Project URL (from POM): https://github.com/RoaringBitmap/RoaringBitmap +License (from POM): Apache 2 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: org.slf4j Name: slf4j-api Version: 2.0.16 +Project URL: http://www.slf4j.org +License (from POM): MIT License - http://www.opensource.org/licenses/mit-license.php + +-------------------------------------------------------------------------------- + +Group: org.xerial Name: sqlite-jdbc Version: 3.47.0.0 +Project URL (from POM): https://github.com/xerial/sqlite-jdbc +License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- \ No newline at end of file diff --git a/open-api/NOTICE b/open-api/NOTICE new file mode 100644 index 000000000000..15e0d63f9135 --- /dev/null +++ b/open-api/NOTICE @@ -0,0 +1,226 @@ + +Apache Iceberg +Copyright 2017-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This project includes code from Kite, developed at Cloudera, Inc. with +the following copyright notice: + +| Copyright 2013 Cloudera Inc. +| +| Licensed 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. + +-------------------------------------------------------------------------------- + +This binary artifact contains code from the following projects: + +-------------------------------------------------------------------------------- +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers. + +## Copyright + +Copyright 2007-, Tatu Saloranta (tatu.saloranta@iki.fi) + +## Licensing + +Jackson 2.x core and extension components are licensed under Apache License 2.0 +To find the details that apply to this artifact see the accompanying LICENSE file. + +## Credits + +A list of contributors may be found from CREDITS(-2.x) file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + +## FastDoubleParser + +jackson-core bundles a shaded copy of FastDoubleParser . +That code is available under an MIT license +under the following copyright. + +Copyright © 2023 Werner Randelshofer, Switzerland. MIT License. + +See FastDoubleParser-NOTICE for details of other source code included in FastDoubleParser +and the licenses and copyrights that apply to that code. + +-------------------------------------------------------------------------------- +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + +Apache Commons Codec +Copyright 2002-2024 The Apache Software Foundation + +Apache Commons IO +Copyright 2002-2024 The Apache Software Foundation + +Apache Avro +Copyright 2009-2024 The Apache Software Foundation + +Apache Commons Compress +Copyright 2002-2024 The Apache Software Foundation + +Apache Commons Lang +Copyright 2001-2023 The Apache Software Foundation + +Apache HttpClient +Copyright 1999-2021 The Apache Software Foundation + +Apache HttpComponents Core HTTP/1.1 +Copyright 2005-2021 The Apache Software Foundation + +Apache HttpComponents Core HTTP/2 +Copyright 2005-2021 The Apache Software Foundation + +-------------------------------------------------------------------------------- + +Apache Hadoop +Copyright 2006 and onwards The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Export Control Notice +--------------------- + +This distribution includes cryptographic software. The country in +which you currently reside may have restrictions on the import, +possession, use, and/or re-export to another country, of +encryption software. BEFORE using any encryption software, please +check your country's laws, regulations and policies concerning the +import, possession, or use, and re-export of encryption software, to +see if this is permitted. See for more +information. + +The U.S. Government Department of Commerce, Bureau of Industry and +Security (BIS), has classified this software as Export Commodity +Control Number (ECCN) 5D002.C.1, which includes information security +software using or performing cryptographic functions with asymmetric +algorithms. The form and manner of this Apache Software Foundation +distribution makes it eligible for export under the License Exception +ENC Technology Software Unrestricted (TSU) exception (see the BIS +Export Administration Regulations, Section 740.13) for both object +code and source code. + +The following provides more details on the included cryptographic software: + +This software uses the SSL libraries from the Jetty project written +by mortbay.org. +Hadoop Yarn Server Web Proxy uses the BouncyCastle Java +cryptography APIs written by the Legion of the Bouncy Castle Inc. + +-------------------------------------------------------------------------------- + +Notices for Eclipse Jetty +========================= +This content is produced and maintained by the Eclipse Jetty project. + +Project home: https://jetty.org/ + +Trademarks +---------- +Eclipse Jetty, and Jetty are trademarks of the Eclipse Foundation. + +Copyright +--------- +All contributions are the property of the respective authors or of +entities to which copyright has been assigned by the authors (eg. employer). + +Declared Project Licenses +------------------------- +This artifacts of this project are made available under the terms of: + + * the Eclipse Public License v2.0 + https://www.eclipse.org/legal/epl-2.0 + SPDX-License-Identifier: EPL-2.0 + + or + + * the Apache License, Version 2.0 + https://www.apache.org/licenses/LICENSE-2.0 + SPDX-License-Identifier: Apache-2.0 + +The following dependencies are EPL. + * org.eclipse.jetty.orbit:org.eclipse.jdt.core + +The following dependencies are EPL and ASL2. + * org.eclipse.jetty.orbit:javax.security.auth.message + +The following dependencies are EPL and CDDL 1.0. + * org.eclipse.jetty.orbit:javax.mail.glassfish + +The following dependencies are CDDL + GPLv2 with classpath exception. +https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + * jakarta.servlet:jakarta.servlet-api + * javax.annotation:javax.annotation-api + * javax.transaction:javax.transaction-api + * javax.websocket:javax.websocket-api + +The following dependencies are licensed by the OW2 Foundation according to the +terms of http://asm.ow2.org/license.html + + * org.ow2.asm:asm-commons + * org.ow2.asm:asm + +The following dependencies are ASL2 licensed. + + * org.apache.taglibs:taglibs-standard-spec + * org.apache.taglibs:taglibs-standard-impl + +The following dependencies are ASL2 licensed. Based on selected classes from +following Apache Tomcat jars, all ASL2 licensed. + + * org.mortbay.jasper:apache-jsp + * org.apache.tomcat:tomcat-jasper + * org.apache.tomcat:tomcat-juli + * org.apache.tomcat:tomcat-jsp-api + * org.apache.tomcat:tomcat-el-api + * org.apache.tomcat:tomcat-jasper-el + * org.apache.tomcat:tomcat-api + * org.apache.tomcat:tomcat-util-scan + * org.apache.tomcat:tomcat-util + * org.mortbay.jasper:apache-el + * org.apache.tomcat:tomcat-jasper-el + * org.apache.tomcat:tomcat-el-api + +The following artifacts are CDDL + GPLv2 with classpath exception. +https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + * org.eclipse.jetty.toolchain:jetty-schemas + +Cryptography +------------ +Content may contain encryption software. The country in which you are currently +may have restrictions on the import, possession, and use, and/or re-export to +another country, of encryption software. BEFORE using any encryption software, +please check the country's laws, regulations and policies concerning the import, +possession, or use, and re-export of encryption software, to see if this is +permitted. + +The UnixCrypt.java code implements the one way cryptography used by +Unix systems for simple password protection. Copyright 1996 Aki Yoshida, +modified April 2001 by Iris Van den Broeke, Daniel Deville. +Permission to use, copy, modify and distribute UnixCrypt +for non-commercial or commercial purposes and without fee is +granted provided that the copyright notice appears in all copies. + +-------------------------------------------------------------------------------- From 7938403a437f81a502fd82053dccff10a1531ada Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 6 Nov 2024 21:32:07 +0100 Subject: [PATCH 109/313] Core, Puffin: Add DV file writer (#11476) --- .../iceberg/deletes/BaseDVFileWriter.java | 194 ++++++++++++++ .../apache/iceberg/deletes/DVFileWriter.java | 46 ++++ .../org/apache/iceberg/io/BaseTaskWriter.java | 3 +- .../apache/iceberg/io/ClusteredWriter.java | 5 +- .../org/apache/iceberg/io/FanoutWriter.java | 3 +- .../org/apache/iceberg/io/StructCopy.java | 8 +- .../org/apache/iceberg/puffin/Puffin.java | 5 + .../apache/iceberg/puffin/PuffinWriter.java | 16 +- .../iceberg/puffin/StandardBlobTypes.java | 3 + .../apache/iceberg/util/StructLikeUtil.java | 67 +++++ .../org/apache/iceberg/io/TestDVWriters.java | 123 +++++++++ .../spark/source/DVWriterBenchmark.java | 251 ++++++++++++++++++ .../spark/source/TestSparkDVWriters.java | 69 +++++ 13 files changed, 786 insertions(+), 7 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/deletes/BaseDVFileWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/deletes/DVFileWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/util/StructLikeUtil.java create mode 100644 data/src/test/java/org/apache/iceberg/io/TestDVWriters.java create mode 100644 spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java diff --git a/core/src/main/java/org/apache/iceberg/deletes/BaseDVFileWriter.java b/core/src/main/java/org/apache/iceberg/deletes/BaseDVFileWriter.java new file mode 100644 index 000000000000..ab47d8ad78d6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/deletes/BaseDVFileWriter.java @@ -0,0 +1,194 @@ +/* + * 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.deletes; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.io.DeleteWriteResult; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.BlobMetadata; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +import org.apache.iceberg.puffin.StandardBlobTypes; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +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.Maps; +import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.StructLikeUtil; + +public class BaseDVFileWriter implements DVFileWriter { + + private static final String REFERENCED_DATA_FILE_KEY = "referenced-data-file"; + private static final String CARDINALITY_KEY = "cardinality"; + + private final OutputFileFactory fileFactory; + private final Function loadPreviousDeletes; + private final Map deletesByPath = Maps.newHashMap(); + private final Map blobsByPath = Maps.newHashMap(); + private DeleteWriteResult result = null; + + public BaseDVFileWriter( + OutputFileFactory fileFactory, Function loadPreviousDeletes) { + this.fileFactory = fileFactory; + this.loadPreviousDeletes = loadPreviousDeletes; + } + + @Override + public void delete(String path, long pos, PartitionSpec spec, StructLike partition) { + Deletes deletes = + deletesByPath.computeIfAbsent(path, key -> new Deletes(path, spec, partition)); + PositionDeleteIndex positions = deletes.positions(); + positions.delete(pos); + } + + @Override + public DeleteWriteResult result() { + Preconditions.checkState(result != null, "Cannot get result from unclosed writer"); + return result; + } + + @Override + public void close() throws IOException { + if (result == null) { + List dvs = Lists.newArrayList(); + CharSequenceSet referencedDataFiles = CharSequenceSet.empty(); + List rewrittenDeleteFiles = Lists.newArrayList(); + + PuffinWriter writer = newWriter(); + + try (PuffinWriter closeableWriter = writer) { + for (Deletes deletes : deletesByPath.values()) { + String path = deletes.path(); + PositionDeleteIndex positions = deletes.positions(); + PositionDeleteIndex previousPositions = loadPreviousDeletes.apply(path); + if (previousPositions != null) { + positions.merge(previousPositions); + for (DeleteFile previousDeleteFile : previousPositions.deleteFiles()) { + // only DVs and file-scoped deletes can be discarded from the table state + if (ContentFileUtil.isFileScoped(previousDeleteFile)) { + rewrittenDeleteFiles.add(previousDeleteFile); + } + } + } + write(closeableWriter, deletes); + referencedDataFiles.add(path); + } + } + + // DVs share the Puffin path and file size but have different offsets + String puffinPath = writer.location(); + long puffinFileSize = writer.fileSize(); + + for (String path : deletesByPath.keySet()) { + DeleteFile dv = createDV(puffinPath, puffinFileSize, path); + dvs.add(dv); + } + + this.result = new DeleteWriteResult(dvs, referencedDataFiles, rewrittenDeleteFiles); + } + } + + private DeleteFile createDV(String path, long size, String referencedDataFile) { + Deletes deletes = deletesByPath.get(referencedDataFile); + BlobMetadata blobMetadata = blobsByPath.get(referencedDataFile); + return FileMetadata.deleteFileBuilder(deletes.spec()) + .ofPositionDeletes() + .withFormat(FileFormat.PUFFIN) + .withPath(path) + .withPartition(deletes.partition()) + .withFileSizeInBytes(size) + .withReferencedDataFile(referencedDataFile) + .withContentOffset(blobMetadata.offset()) + .withContentSizeInBytes(blobMetadata.length()) + .withRecordCount(deletes.positions().cardinality()) + .build(); + } + + private void write(PuffinWriter writer, Deletes deletes) { + String path = deletes.path(); + PositionDeleteIndex positions = deletes.positions(); + BlobMetadata blobMetadata = writer.write(toBlob(positions, path)); + blobsByPath.put(path, blobMetadata); + } + + private PuffinWriter newWriter() { + EncryptedOutputFile outputFile = fileFactory.newOutputFile(); + String ident = "Iceberg " + IcebergBuild.fullVersion(); + return Puffin.write(outputFile).createdBy(ident).build(); + } + + private Blob toBlob(PositionDeleteIndex positions, String path) { + return new Blob( + StandardBlobTypes.DV_V1, + ImmutableList.of(MetadataColumns.ROW_POSITION.fieldId()), + -1 /* snapshot ID is inherited */, + -1 /* sequence number is inherited */, + positions.serialize(), + null /* uncompressed */, + ImmutableMap.of( + REFERENCED_DATA_FILE_KEY, + path, + CARDINALITY_KEY, + String.valueOf(positions.cardinality()))); + } + + private static class Deletes { + private final String path; + private final PositionDeleteIndex positions; + private final PartitionSpec spec; + private final StructLike partition; + + private Deletes(String path, PartitionSpec spec, StructLike partition) { + this.path = path; + this.positions = new BitmapPositionDeleteIndex(); + this.spec = spec; + this.partition = StructLikeUtil.copy(partition); + } + + public String path() { + return path; + } + + public PositionDeleteIndex positions() { + return positions; + } + + public PartitionSpec spec() { + return spec; + } + + public StructLike partition() { + return partition; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/deletes/DVFileWriter.java b/core/src/main/java/org/apache/iceberg/deletes/DVFileWriter.java new file mode 100644 index 000000000000..2561f7be3d34 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/deletes/DVFileWriter.java @@ -0,0 +1,46 @@ +/* + * 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.deletes; + +import java.io.Closeable; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.io.DeleteWriteResult; + +/** A deletion vector file writer. */ +public interface DVFileWriter extends Closeable { + /** + * Marks a position in a given data file as deleted. + * + * @param path the data file path + * @param pos the data file position + * @param spec the data file partition spec + * @param partition the data file partition + */ + void delete(String path, long pos, PartitionSpec spec, StructLike partition); + + /** + * Returns a result that contains information about written {@link DeleteFile}s. The result is + * valid only after the writer is closed. + * + * @return the writer result + */ + DeleteWriteResult result(); +} diff --git a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index 471dc3e56035..968db0ab538b 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.StructLikeMap; +import org.apache.iceberg.util.StructLikeUtil; import org.apache.iceberg.util.StructProjection; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; @@ -149,7 +150,7 @@ public void write(T row) throws IOException { PathOffset pathOffset = PathOffset.of(dataWriter.currentPath(), dataWriter.currentRows()); // Create a copied key from this row. - StructLike copiedKey = StructCopy.copy(structProjection.wrap(asStructLike(row))); + StructLike copiedKey = StructLikeUtil.copy(structProjection.wrap(asStructLike(row))); // Adding a pos-delete to replace the old path-offset. PathOffset previous = insertedRowMap.put(copiedKey, pathOffset); diff --git a/core/src/main/java/org/apache/iceberg/io/ClusteredWriter.java b/core/src/main/java/org/apache/iceberg/io/ClusteredWriter.java index 1dc4871f0a12..06093ff2a943 100644 --- a/core/src/main/java/org/apache/iceberg/io/ClusteredWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/ClusteredWriter.java @@ -29,6 +29,7 @@ import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.StructLikeSet; +import org.apache.iceberg.util.StructLikeUtil; /** * A writer capable of writing to multiple specs and partitions that requires the incoming records @@ -81,7 +82,7 @@ public void write(T row, PartitionSpec spec, StructLike partition) { this.partitionComparator = Comparators.forType(partitionType); this.completedPartitions = StructLikeSet.create(partitionType); // copy the partition key as the key object may be reused - this.currentPartition = StructCopy.copy(partition); + this.currentPartition = StructLikeUtil.copy(partition); this.currentWriter = newWriter(currentSpec, currentPartition); } else if (partition != currentPartition @@ -96,7 +97,7 @@ public void write(T row, PartitionSpec spec, StructLike partition) { } // copy the partition key as the key object may be reused - this.currentPartition = StructCopy.copy(partition); + this.currentPartition = StructLikeUtil.copy(partition); this.currentWriter = newWriter(currentSpec, currentPartition); } diff --git a/core/src/main/java/org/apache/iceberg/io/FanoutWriter.java b/core/src/main/java/org/apache/iceberg/io/FanoutWriter.java index 95d5cc1afcc9..340f4b3558d9 100644 --- a/core/src/main/java/org/apache/iceberg/io/FanoutWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/FanoutWriter.java @@ -25,6 +25,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.StructLikeMap; +import org.apache.iceberg.util.StructLikeUtil; /** * A writer capable of writing to multiple specs and partitions that keeps files for each seen @@ -59,7 +60,7 @@ private FileWriter writer(PartitionSpec spec, StructLike partition) { if (writer == null) { // copy the partition key as the key object may be reused - StructLike copiedPartition = StructCopy.copy(partition); + StructLike copiedPartition = StructLikeUtil.copy(partition); writer = newWriter(spec, copiedPartition); specWriters.put(copiedPartition, writer); } diff --git a/core/src/main/java/org/apache/iceberg/io/StructCopy.java b/core/src/main/java/org/apache/iceberg/io/StructCopy.java index 229dff371762..f8adbf62a1d6 100644 --- a/core/src/main/java/org/apache/iceberg/io/StructCopy.java +++ b/core/src/main/java/org/apache/iceberg/io/StructCopy.java @@ -20,7 +20,13 @@ import org.apache.iceberg.StructLike; -/** Copy the StructLike's values into a new one. It does not handle list or map values now. */ +/** + * Copy the StructLike's values into a new one. It does not handle list or map values now. + * + * @deprecated since 1.8.0, will be removed in 1.9.0; use {@link + * org.apache.iceberg.util.StructLikeUtil#copy(StructLike)} instead. + */ +@Deprecated class StructCopy implements StructLike { static StructLike copy(StructLike struct) { return struct != null ? new StructCopy(struct) : null; diff --git a/core/src/main/java/org/apache/iceberg/puffin/Puffin.java b/core/src/main/java/org/apache/iceberg/puffin/Puffin.java index 251486d01e76..d20a5596c4d4 100644 --- a/core/src/main/java/org/apache/iceberg/puffin/Puffin.java +++ b/core/src/main/java/org/apache/iceberg/puffin/Puffin.java @@ -19,6 +19,7 @@ package org.apache.iceberg.puffin; import java.util.Map; +import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -31,6 +32,10 @@ public static WriteBuilder write(OutputFile outputFile) { return new WriteBuilder(outputFile); } + public static WriteBuilder write(EncryptedOutputFile outputFile) { + return new WriteBuilder(outputFile.encryptingOutputFile()); + } + /** A builder for {@link PuffinWriter}. */ public static class WriteBuilder { private final OutputFile outputFile; diff --git a/core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java b/core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java index 5728b7474885..cd44dab03b90 100644 --- a/core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java +++ b/core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java @@ -44,6 +44,7 @@ public class PuffinWriter implements FileAppender { // Must not be modified private static final byte[] MAGIC = PuffinFormat.getMagic(); + private final OutputFile outputFile; private final PositionOutputStream outputStream; private final Map properties; private final PuffinCompressionCodec footerCompression; @@ -63,6 +64,7 @@ public class PuffinWriter implements FileAppender { Preconditions.checkNotNull(outputFile, "outputFile is null"); Preconditions.checkNotNull(properties, "properties is null"); Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null"); + this.outputFile = outputFile; this.outputStream = outputFile.create(); this.properties = ImmutableMap.copyOf(properties); this.footerCompression = @@ -70,8 +72,16 @@ public class PuffinWriter implements FileAppender { this.defaultBlobCompression = defaultBlobCompression; } + public String location() { + return outputFile.location(); + } + @Override public void add(Blob blob) { + write(blob); + } + + public BlobMetadata write(Blob blob) { Preconditions.checkNotNull(blob, "blob is null"); checkNotFinished(); try { @@ -82,7 +92,7 @@ public void add(Blob blob) { ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData()); int length = rawData.remaining(); IOUtil.writeFully(outputStream, rawData); - writtenBlobsMetadata.add( + BlobMetadata blobMetadata = new BlobMetadata( blob.type(), blob.inputFields(), @@ -91,7 +101,9 @@ public void add(Blob blob) { fileOffset, length, codec.codecName(), - blob.properties())); + blob.properties()); + writtenBlobsMetadata.add(blobMetadata); + return blobMetadata; } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/core/src/main/java/org/apache/iceberg/puffin/StandardBlobTypes.java b/core/src/main/java/org/apache/iceberg/puffin/StandardBlobTypes.java index 2be5df5f88b9..ce78375c4b1a 100644 --- a/core/src/main/java/org/apache/iceberg/puffin/StandardBlobTypes.java +++ b/core/src/main/java/org/apache/iceberg/puffin/StandardBlobTypes.java @@ -26,4 +26,7 @@ private StandardBlobTypes() {} * href="https://datasketches.apache.org/">Apache DataSketches library */ public static final String APACHE_DATASKETCHES_THETA_V1 = "apache-datasketches-theta-v1"; + + /** A serialized deletion vector according to the Iceberg spec */ + public static final String DV_V1 = "deletion-vector-v1"; } diff --git a/core/src/main/java/org/apache/iceberg/util/StructLikeUtil.java b/core/src/main/java/org/apache/iceberg/util/StructLikeUtil.java new file mode 100644 index 000000000000..5285793a4aad --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/StructLikeUtil.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import org.apache.iceberg.StructLike; + +public class StructLikeUtil { + + private StructLikeUtil() {} + + public static StructLike copy(StructLike struct) { + return StructCopy.copy(struct); + } + + private static class StructCopy implements StructLike { + private static StructLike copy(StructLike struct) { + return struct != null ? new StructCopy(struct) : null; + } + + private final Object[] values; + + private StructCopy(StructLike toCopy) { + this.values = new Object[toCopy.size()]; + + for (int i = 0; i < values.length; i += 1) { + Object value = toCopy.get(i, Object.class); + + if (value instanceof StructLike) { + values[i] = copy((StructLike) value); + } else { + values[i] = value; + } + } + } + + @Override + public int size() { + return values.length; + } + + @Override + public T get(int pos, Class javaClass) { + return javaClass.cast(values[pos]); + } + + @Override + public void set(int pos, T value) { + throw new UnsupportedOperationException("Struct copy cannot be modified"); + } + } +} diff --git a/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java b/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java new file mode 100644 index 000000000000..ce742b1c4685 --- /dev/null +++ b/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java @@ -0,0 +1,123 @@ +/* + * 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.io; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.StructLikeSet; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestDVWriters extends WriterTestBase { + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(new Object[] {3}); + } + + private OutputFileFactory fileFactory = null; + + protected abstract StructLikeSet toSet(Iterable records); + + protected FileFormat dataFormat() { + return FileFormat.PARQUET; + } + + @Override + @BeforeEach + public void setupTable() throws Exception { + this.table = create(SCHEMA, PartitionSpec.unpartitioned()); + this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PUFFIN).build(); + } + + @TestTemplate + public void testBasicDVs() throws IOException { + FileWriterFactory writerFactory = newWriterFactory(table.schema()); + + // add the first data file + List rows1 = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(11, "aaa")); + DataFile dataFile1 = writeData(writerFactory, fileFactory, rows1, table.spec(), null); + table.newFastAppend().appendFile(dataFile1).commit(); + + // add the second data file + List rows2 = ImmutableList.of(toRow(3, "aaa"), toRow(4, "aaa"), toRow(12, "aaa")); + DataFile dataFile2 = writeData(writerFactory, fileFactory, rows2, table.spec(), null); + table.newFastAppend().appendFile(dataFile2).commit(); + + // init the DV writer + DVFileWriter dvWriter = + new BaseDVFileWriter(fileFactory, new PreviousDeleteLoader(table, ImmutableMap.of())); + + // write deletes for both data files (the order of records is mixed) + dvWriter.delete(dataFile1.location(), 1L, table.spec(), null); + dvWriter.delete(dataFile2.location(), 0L, table.spec(), null); + dvWriter.delete(dataFile1.location(), 0L, table.spec(), null); + dvWriter.delete(dataFile2.location(), 1L, table.spec(), null); + dvWriter.close(); + + // verify the writer result + DeleteWriteResult result = dvWriter.result(); + assertThat(result.deleteFiles()).hasSize(2); + assertThat(result.referencedDataFiles()) + .hasSize(2) + .contains(dataFile1.location()) + .contains(dataFile2.location()); + assertThat(result.referencesDataFiles()).isTrue(); + } + + private static class PreviousDeleteLoader implements Function { + private final Map deleteFiles; + private final DeleteLoader deleteLoader; + + PreviousDeleteLoader(Table table, Map deleteFiles) { + this.deleteFiles = deleteFiles; + this.deleteLoader = new BaseDeleteLoader(deleteFile -> table.io().newInputFile(deleteFile)); + } + + @Override + public PositionDeleteIndex apply(String path) { + DeleteFile deleteFile = deleteFiles.get(path); + if (deleteFile == null) { + return null; + } + return deleteLoader.loadPositionDeletes(ImmutableList.of(deleteFile), path); + } + } +} diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java new file mode 100644 index 000000000000..ac74fb5a109c --- /dev/null +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.io.DeleteWriteResult; +import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.unsafe.types.UTF8String; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that compares the performance of DV and position delete writers. + * + *

    To run this benchmark for spark-3.5: + * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5_2.12:jmh + * -PjmhIncludeRegex=DVWriterBenchmark + * -PjmhOutputPath=benchmark/iceberg-dv-writer-benchmark-result.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 10) +@Timeout(time = 20, timeUnit = TimeUnit.MINUTES) +@BenchmarkMode(Mode.SingleShotTime) +public class DVWriterBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final int DATA_FILE_RECORD_COUNT = 2_000_000; + private static final long TARGET_FILE_SIZE = Long.MAX_VALUE; + + @Param({"5", "10"}) + private int referencedDataFileCount; + + @Param({"0.01", "0.03", "0.05", "0.10", "0.2"}) + private double deletedRowsRatio; + + private final Configuration hadoopConf = new Configuration(); + private final Random random = ThreadLocalRandom.current(); + private SparkSession spark; + private Table table; + private Iterable positionDeletes; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException { + setupSpark(); + initTable(); + generatePositionDeletes(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + tearDownSpark(); + } + + @Benchmark + @Threads(1) + public void dv(Blackhole blackhole) throws IOException { + OutputFileFactory fileFactory = newFileFactory(FileFormat.PUFFIN); + DVFileWriter writer = new BaseDVFileWriter(fileFactory, path -> null); + + try (DVFileWriter closableWriter = writer) { + for (InternalRow row : positionDeletes) { + String path = row.getString(0); + long pos = row.getLong(1); + closableWriter.delete(path, pos, table.spec(), null); + } + } + + DeleteWriteResult result = writer.result(); + blackhole.consume(result); + } + + @Benchmark + @Threads(1) + public void fileScopedParquetDeletes(Blackhole blackhole) throws IOException { + FanoutPositionOnlyDeleteWriter writer = newWriter(DeleteGranularity.FILE); + write(writer, positionDeletes); + DeleteWriteResult result = writer.result(); + blackhole.consume(result); + } + + @Benchmark + @Threads(1) + public void partitionScopedParquetDeletes(Blackhole blackhole) throws IOException { + FanoutPositionOnlyDeleteWriter writer = newWriter(DeleteGranularity.PARTITION); + write(writer, positionDeletes); + DeleteWriteResult result = writer.result(); + blackhole.consume(result); + } + + private FanoutPositionOnlyDeleteWriter newWriter(DeleteGranularity granularity) { + return new FanoutPositionOnlyDeleteWriter<>( + newWriterFactory(), + newFileFactory(FileFormat.PARQUET), + table.io(), + TARGET_FILE_SIZE, + granularity); + } + + private DeleteWriteResult write( + FanoutPositionOnlyDeleteWriter writer, Iterable rows) + throws IOException { + + try (FanoutPositionOnlyDeleteWriter closableWriter = writer) { + PositionDelete positionDelete = PositionDelete.create(); + + for (InternalRow row : rows) { + String path = row.getString(0); + long pos = row.getLong(1); + positionDelete.set(path, pos, null /* no row */); + closableWriter.write(positionDelete, table.spec(), null); + } + } + + return writer.result(); + } + + private SparkFileWriterFactory newWriterFactory() { + return SparkFileWriterFactory.builderFor(table).dataFileFormat(FileFormat.PARQUET).build(); + } + + private OutputFileFactory newFileFactory(FileFormat format) { + return OutputFileFactory.builderFor(table, 1, 1).format(format).build(); + } + + private void generatePositionDeletes() { + int numDeletesPerFile = (int) (DATA_FILE_RECORD_COUNT * deletedRowsRatio); + int numDeletes = referencedDataFileCount * numDeletesPerFile; + List deletes = Lists.newArrayListWithExpectedSize(numDeletes); + + for (int pathIndex = 0; pathIndex < referencedDataFileCount; pathIndex++) { + UTF8String dataFilePath = UTF8String.fromString(generateDataFilePath()); + Set positions = generatePositions(numDeletesPerFile); + for (long pos : positions) { + deletes.add(new GenericInternalRow(new Object[] {dataFilePath, pos})); + } + } + + Collections.shuffle(deletes); + + this.positionDeletes = deletes; + } + + public Set generatePositions(int numPositions) { + Set positions = Sets.newHashSet(); + + while (positions.size() < numPositions) { + long pos = random.nextInt(DATA_FILE_RECORD_COUNT); + positions.add(pos); + } + + return positions; + } + + private String generateDataFilePath() { + String fileName = FileGenerationUtil.generateFileName(); + return table.locationProvider().newDataLocation(table.spec(), null, fileName); + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .master("local[*]") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (c1 INT, c2 INT, c3 STRING) USING iceberg", TABLE_NAME); + this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java new file mode 100644 index 000000000000..dfc693d3094d --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestDVWriters; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +public class TestSparkDVWriters extends TestDVWriters { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return SparkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(dataFormat()) + .deleteFileFormat(dataFormat()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected InternalRow toRow(Integer id, String data) { + InternalRow row = new GenericInternalRow(2); + row.update(0, id); + row.update(1, UTF8String.fromString(data)); + return row; + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + StructType sparkType = SparkSchemaUtil.convert(table.schema()); + for (InternalRow row : rows) { + InternalRowWrapper wrapper = new InternalRowWrapper(sparkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} From 1e82c476df58ba467342eec98cb02ec0e2c75bd1 Mon Sep 17 00:00:00 2001 From: Mingliang Liu Date: Wed, 6 Nov 2024 14:45:41 -0800 Subject: [PATCH 110/313] Flink: Fix config key typo in error message of SplitComparators (#11482) --- .../apache/iceberg/flink/source/split/SplitComparators.java | 4 +++- .../assigner/TestFileSequenceNumberBasedSplitAssigner.java | 2 +- .../apache/iceberg/flink/source/split/SplitComparators.java | 4 +++- .../assigner/TestFileSequenceNumberBasedSplitAssigner.java | 2 +- .../apache/iceberg/flink/source/split/SplitComparators.java | 4 +++- .../assigner/TestFileSequenceNumberBasedSplitAssigner.java | 2 +- 6 files changed, 12 insertions(+), 6 deletions(-) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index 56ee92014d12..37bddfbb7182 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.split; +import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -35,7 +36,8 @@ public static SerializableComparator fileSequenceNumber() { return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { Preconditions.checkArgument( o1.task().files().size() == 1 && o2.task().files().size() == 1, - "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split"); + "Could not compare combined task. Please use '%s' to prevent combining multiple files to a split", + FlinkReadOptions.SPLIT_FILE_OPEN_COST); Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java index ff63ba8e58a0..2b65977fb2f9 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -43,7 +43,7 @@ public void testMultipleFilesInAnIcebergSplit() { () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Please use 'split-open-file-cost'"); + .hasMessageContaining("Please use 'split-file-open-cost'"); } /** Test sorted splits */ diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index 56ee92014d12..37bddfbb7182 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.split; +import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -35,7 +36,8 @@ public static SerializableComparator fileSequenceNumber() { return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { Preconditions.checkArgument( o1.task().files().size() == 1 && o2.task().files().size() == 1, - "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split"); + "Could not compare combined task. Please use '%s' to prevent combining multiple files to a split", + FlinkReadOptions.SPLIT_FILE_OPEN_COST); Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java index ff63ba8e58a0..2b65977fb2f9 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -43,7 +43,7 @@ public void testMultipleFilesInAnIcebergSplit() { () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Please use 'split-open-file-cost'"); + .hasMessageContaining("Please use 'split-file-open-cost'"); } /** Test sorted splits */ diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index 56ee92014d12..37bddfbb7182 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.split; +import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -35,7 +36,8 @@ public static SerializableComparator fileSequenceNumber() { return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { Preconditions.checkArgument( o1.task().files().size() == 1 && o2.task().files().size() == 1, - "Could not compare combined task. Please use 'split-open-file-cost' to prevent combining multiple files to a split"); + "Could not compare combined task. Please use '%s' to prevent combining multiple files to a split", + FlinkReadOptions.SPLIT_FILE_OPEN_COST); Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java index ff63ba8e58a0..2b65977fb2f9 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -43,7 +43,7 @@ public void testMultipleFilesInAnIcebergSplit() { () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Please use 'split-open-file-cost'"); + .hasMessageContaining("Please use 'split-file-open-cost'"); } /** Test sorted splits */ From 11e7230b678c1dd76993117c7470ce92b63709d7 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 7 Nov 2024 09:47:50 -0600 Subject: [PATCH 111/313] API: Removes Explicit Parameterization of Schema Tests (#11444) --- .../main/java/org/apache/iceberg/Schema.java | 8 +- .../java/org/apache/iceberg/TestHelpers.java | 3 + .../java/org/apache/iceberg/TestSchema.java | 119 ++++++++++++------ 3 files changed, 93 insertions(+), 37 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index 44f65ff56a54..a94e8771875a 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.BiMap; @@ -54,8 +55,11 @@ public class Schema implements Serializable { private static final Joiner NEWLINE = Joiner.on('\n'); private static final String ALL_COLUMNS = "*"; private static final int DEFAULT_SCHEMA_ID = 0; - private static final int DEFAULT_VALUES_MIN_FORMAT_VERSION = 3; - private static final Map MIN_FORMAT_VERSIONS = + + @VisibleForTesting static final int DEFAULT_VALUES_MIN_FORMAT_VERSION = 3; + + @VisibleForTesting + static final Map MIN_FORMAT_VERSIONS = ImmutableMap.of(Type.TypeID.TIMESTAMP_NANO, 3); private final StructType struct; diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index ca3b1a908ac6..003e7835ed4b 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -53,6 +53,9 @@ public class TestHelpers { private TestHelpers() {} + public static final int MAX_FORMAT_VERSION = 3; + public static final int[] ALL_VERSIONS = IntStream.rangeClosed(1, MAX_FORMAT_VERSION).toArray(); + /** Wait in a tight check loop until system clock is past {@code timestampMillis} */ public static long waitUntilAfter(long timestampMillis) { long current = System.currentTimeMillis(); diff --git a/api/src/test/java/org/apache/iceberg/TestSchema.java b/api/src/test/java/org/apache/iceberg/TestSchema.java index fec7343c5cbc..e79adbd09fb7 100644 --- a/api/src/test/java/org/apache/iceberg/TestSchema.java +++ b/api/src/test/java/org/apache/iceberg/TestSchema.java @@ -18,32 +18,27 @@ */ package org.apache.iceberg; +import static org.apache.iceberg.Schema.DEFAULT_VALUES_MIN_FORMAT_VERSION; +import static org.apache.iceberg.Schema.MIN_FORMAT_VERSIONS; +import static org.apache.iceberg.TestHelpers.MAX_FORMAT_VERSION; import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.util.List; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.FieldSource; +import org.junit.jupiter.params.provider.MethodSource; public class TestSchema { - private static final Schema TS_NANO_CASES = - new Schema( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "ts", Types.TimestampNanoType.withZone()), - Types.NestedField.optional( - 3, "arr", Types.ListType.ofRequired(4, Types.TimestampNanoType.withoutZone())), - Types.NestedField.required( - 5, - "struct", - Types.StructType.of( - Types.NestedField.optional(6, "inner_ts", Types.TimestampNanoType.withZone()), - Types.NestedField.required(7, "data", Types.StringType.get()))), - Types.NestedField.optional( - 8, - "struct_arr", - Types.StructType.of( - Types.NestedField.optional(9, "ts", Types.TimestampNanoType.withoutZone())))); + + private static final List TEST_TYPES = + ImmutableList.of(Types.TimestampNanoType.withoutZone(), Types.TimestampNanoType.withZone()); private static final Schema INITIAL_DEFAULT_SCHEMA = new Schema( @@ -64,27 +59,77 @@ public class TestSchema { .withWriteDefault("--") .build()); + private Schema generateTypeSchema(Type type) { + return new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "top", type), + Types.NestedField.optional(3, "arr", Types.ListType.ofRequired(4, type)), + Types.NestedField.required( + 5, + "struct", + Types.StructType.of( + Types.NestedField.optional(6, "inner_op", type), + Types.NestedField.required(7, "inner_req", type), + Types.NestedField.optional( + 8, + "struct_arr", + Types.StructType.of(Types.NestedField.optional(9, "deep", type)))))); + } + + private static Stream unsupportedTypes() { + return TEST_TYPES.stream() + .flatMap( + type -> + IntStream.range(1, MIN_FORMAT_VERSIONS.get(type.typeId())) + .mapToObj(unsupportedVersion -> Arguments.of(type, unsupportedVersion))); + } + @ParameterizedTest - @ValueSource(ints = {1, 2}) - public void testUnsupportedTimestampNano(int formatVersion) { - assertThatThrownBy(() -> Schema.checkCompatibility(TS_NANO_CASES, formatVersion)) + @MethodSource("unsupportedTypes") + public void testUnsupportedTypes(Type type, int unsupportedVersion) { + assertThatThrownBy( + () -> Schema.checkCompatibility(generateTypeSchema(type), unsupportedVersion)) .isInstanceOf(IllegalStateException.class) .hasMessage( "Invalid schema for v%s:\n" - + "- Invalid type for ts: timestamptz_ns is not supported until v3\n" - + "- Invalid type for arr.element: timestamp_ns is not supported until v3\n" - + "- Invalid type for struct.inner_ts: timestamptz_ns is not supported until v3\n" - + "- Invalid type for struct_arr.ts: timestamp_ns is not supported until v3", - formatVersion); + + "- Invalid type for top: %s is not supported until v%s\n" + + "- Invalid type for arr.element: %s is not supported until v%s\n" + + "- Invalid type for struct.inner_op: %s is not supported until v%s\n" + + "- Invalid type for struct.inner_req: %s is not supported until v%s\n" + + "- Invalid type for struct.struct_arr.deep: %s is not supported until v%s", + unsupportedVersion, + type, + MIN_FORMAT_VERSIONS.get(type.typeId()), + type, + MIN_FORMAT_VERSIONS.get(type.typeId()), + type, + MIN_FORMAT_VERSIONS.get(type.typeId()), + type, + MIN_FORMAT_VERSIONS.get(type.typeId()), + type, + MIN_FORMAT_VERSIONS.get(type.typeId())); } - @Test - public void testSupportedTimestampNano() { - assertThatCode(() -> Schema.checkCompatibility(TS_NANO_CASES, 3)).doesNotThrowAnyException(); + private static Stream supportedTypes() { + return TEST_TYPES.stream() + .flatMap( + type -> + IntStream.rangeClosed(MIN_FORMAT_VERSIONS.get(type.typeId()), MAX_FORMAT_VERSION) + .mapToObj(supportedVersion -> Arguments.of(type, supportedVersion))); } @ParameterizedTest - @ValueSource(ints = {1, 2}) + @MethodSource("supportedTypes") + public void testTypeSupported(Type type, int supportedVersion) { + assertThatCode(() -> Schema.checkCompatibility(generateTypeSchema(type), supportedVersion)) + .doesNotThrowAnyException(); + } + + private static int[] unsupportedInitialDefault = + IntStream.range(1, DEFAULT_VALUES_MIN_FORMAT_VERSION).toArray(); + + @ParameterizedTest + @FieldSource("unsupportedInitialDefault") public void testUnsupportedInitialDefault(int formatVersion) { assertThatThrownBy(() -> Schema.checkCompatibility(INITIAL_DEFAULT_SCHEMA, formatVersion)) .isInstanceOf(IllegalStateException.class) @@ -95,14 +140,18 @@ public void testUnsupportedInitialDefault(int formatVersion) { formatVersion); } - @Test - public void testSupportedInitialDefault() { - assertThatCode(() -> Schema.checkCompatibility(INITIAL_DEFAULT_SCHEMA, 3)) + private static int[] supportedInitialDefault = + IntStream.rangeClosed(DEFAULT_VALUES_MIN_FORMAT_VERSION, MAX_FORMAT_VERSION).toArray(); + + @ParameterizedTest + @FieldSource("supportedInitialDefault") + public void testSupportedInitialDefault(int formatVersion) { + assertThatCode(() -> Schema.checkCompatibility(INITIAL_DEFAULT_SCHEMA, formatVersion)) .doesNotThrowAnyException(); } @ParameterizedTest - @ValueSource(ints = {1, 2, 3}) + @FieldSource("org.apache.iceberg.TestHelpers#ALL_VERSIONS") public void testSupportedWriteDefault(int formatVersion) { // only the initial default is a forward-incompatible change assertThatCode(() -> Schema.checkCompatibility(WRITE_DEFAULT_SCHEMA, formatVersion)) From 5c8a5d68d43bc9d32fe5cb272be7ea01b95a7251 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 8 Nov 2024 00:50:10 +0800 Subject: [PATCH 112/313] Docs: Fix verifying release candidate with Spark and Flink (#11461) --- site/docs/how-to-release.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index 6befe74b78de..85ecd64e51f7 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -422,11 +422,11 @@ spark-runtime jar for the Spark installation): ```bash spark-shell \ --conf spark.jars.repositories=${MAVEN_URL} \ - --packages org.apache.iceberg:iceberg-spark3-runtime:{{ icebergVersion }} \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:{{ icebergVersion }} \ --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \ --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ --conf spark.sql.catalog.local.type=hadoop \ - --conf spark.sql.catalog.local.warehouse=${LOCAL_WAREHOUSE_PATH} \ + --conf spark.sql.catalog.local.warehouse=$PWD/warehouse \ --conf spark.sql.catalog.local.default-namespace=default \ --conf spark.sql.defaultCatalog=local ``` @@ -435,13 +435,13 @@ spark-shell \ To verify using Flink, start a Flink SQL Client with the following command: ```bash -wget ${MAVEN_URL}/iceberg-flink-runtime/{{ icebergVersion }}/iceberg-flink-runtime-{{ icebergVersion }}.jar +wget ${MAVEN_URL}/iceberg-flink-runtime-1.20/{{ icebergVersion }}/iceberg-flink-runtime-1.20-{{ icebergVersion }}.jar sql-client.sh embedded \ - -j iceberg-flink-runtime-{{ icebergVersion }}.jar \ - -j ${FLINK_CONNECTOR_PACKAGE}-${HIVE_VERSION}_${SCALA_VERSION}-${FLINK_VERSION}.jar \ + -j iceberg-flink-runtime-1.20-{{ icebergVersion }}.jar \ + -j flink-connector-hive_2.12-1.20.jar \ shell -``` + ## Voting From 3da64d37d018b3fc7e78e7004ce63245b5b8a1c5 Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 8 Nov 2024 07:29:44 +0100 Subject: [PATCH 113/313] Flink: Port #11144 to v1.19 (#11473) --- .../maintenance/api/ExpireSnapshots.java | 125 +++++ .../{operator => api}/JdbcLockFactory.java | 7 +- .../api/MaintenanceTaskBuilder.java | 227 +++++++++ .../maintenance/api/TableMaintenance.java | 330 +++++++++++++ .../{operator => api}/TaskResult.java | 7 +- .../{operator => api}/Trigger.java | 31 +- .../{operator => api}/TriggerLockFactory.java | 9 +- .../operator/DeleteFilesProcessor.java | 130 +++++ .../operator/ExpireSnapshotsProcessor.java | 129 +++++ .../maintenance/operator/LockRemover.java | 25 +- .../maintenance/operator/MonitorSource.java | 4 +- .../maintenance/operator/TableChange.java | 10 +- .../operator/TableMaintenanceMetrics.java | 10 +- .../operator/TriggerEvaluator.java | 8 +- .../maintenance/operator/TriggerManager.java | 54 +- .../api/MaintenanceTaskInfraExtension.java | 78 +++ .../api/MaintenanceTaskTestBase.java | 64 +++ .../maintenance/api/TestExpireSnapshots.java | 254 ++++++++++ .../TestJdbcLockFactory.java | 4 +- .../TestLockFactoryBase.java | 2 +- .../maintenance/api/TestMaintenanceE2E.java | 67 +++ .../maintenance/api/TestTableMaintenance.java | 460 ++++++++++++++++++ .../maintenance/operator/CollectingSink.java | 6 +- .../operator/ConstantsForTests.java | 29 -- .../operator/FlinkSqlExtension.java | 135 ----- .../operator/FlinkStreamingTestUtils.java | 73 --- .../maintenance/operator/ManualSource.java | 12 +- .../MetricsReporterFactoryForTests.java | 65 ++- .../operator/OperatorTestBase.java | 184 +++++-- .../operator/TestDeleteFilesProcessor.java | 116 +++++ .../TestExpireSnapshotsProcessor.java | 80 +++ .../maintenance/operator/TestLockRemover.java | 120 +++-- .../operator/TestMonitorSource.java | 72 +-- .../operator/TestTriggerManager.java | 152 +++--- 34 files changed, 2527 insertions(+), 552 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java rename flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/JdbcLockFactory.java (98%) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java rename flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/TaskResult.java (93%) rename flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/Trigger.java (67%) rename flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/{operator => api}/TriggerLockFactory.java (86%) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskInfraExtension.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java rename flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/{operator => api}/TestJdbcLockFactory.java (92%) rename flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/{operator => api}/TestLockFactoryBase.java (97%) create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java new file mode 100644 index 000000000000..9cde5cb173e1 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java @@ -0,0 +1,125 @@ +/* + * 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.flink.maintenance.api; + +import java.time.Duration; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.flink.maintenance.operator.DeleteFilesProcessor; +import org.apache.iceberg.flink.maintenance.operator.ExpireSnapshotsProcessor; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Deletes expired snapshots and the corresponding files. */ +public class ExpireSnapshots { + private static final int DELETE_BATCH_SIZE_DEFAULT = 1000; + private static final String EXECUTOR_OPERATOR_NAME = "Expire Snapshot"; + @VisibleForTesting static final String DELETE_FILES_OPERATOR_NAME = "Delete file"; + + private ExpireSnapshots() {} + + /** Creates the builder for creating a stream which expires snapshots for the table. */ + public static Builder builder() { + return new Builder(); + } + + public static class Builder extends MaintenanceTaskBuilder { + private Duration maxSnapshotAge = null; + private Integer numSnapshots = null; + private Integer planningWorkerPoolSize; + private int deleteBatchSize = DELETE_BATCH_SIZE_DEFAULT; + + /** + * The snapshots older than this age will be removed. + * + * @param newMaxSnapshotAge of the snapshots to be removed + */ + public Builder maxSnapshotAge(Duration newMaxSnapshotAge) { + this.maxSnapshotAge = newMaxSnapshotAge; + return this; + } + + /** + * The minimum number of {@link Snapshot}s to retain. For more details description see {@link + * org.apache.iceberg.ExpireSnapshots#retainLast(int)}. + * + * @param newNumSnapshots number of snapshots to retain + */ + public Builder retainLast(int newNumSnapshots) { + this.numSnapshots = newNumSnapshots; + return this; + } + + /** + * The worker pool size used to calculate the files to delete. If not set, the shared worker + * pool is used. + * + * @param newPlanningWorkerPoolSize for planning files to delete + */ + public Builder planningWorkerPoolSize(int newPlanningWorkerPoolSize) { + this.planningWorkerPoolSize = newPlanningWorkerPoolSize; + return this; + } + + /** + * Size of the batch used to deleting the files. + * + * @param newDeleteBatchSize used for deleting + */ + public Builder deleteBatchSize(int newDeleteBatchSize) { + this.deleteBatchSize = newDeleteBatchSize; + return this; + } + + @Override + DataStream append(DataStream trigger) { + Preconditions.checkNotNull(tableLoader(), "TableLoader should not be null"); + + SingleOutputStreamOperator result = + trigger + .process( + new ExpireSnapshotsProcessor( + tableLoader(), + maxSnapshotAge == null ? null : maxSnapshotAge.toMillis(), + numSnapshots, + planningWorkerPoolSize)) + .name(operatorName(EXECUTOR_OPERATOR_NAME)) + .uid(EXECUTOR_OPERATOR_NAME + uidSuffix()) + .slotSharingGroup(slotSharingGroup()) + .forceNonParallel(); + + result + .getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM) + .rebalance() + .transform( + operatorName(DELETE_FILES_OPERATOR_NAME), + TypeInformation.of(Void.class), + new DeleteFilesProcessor( + index(), taskName(), tableLoader().loadTable(), deleteBatchSize)) + .uid(DELETE_FILES_OPERATOR_NAME + uidSuffix()) + .slotSharingGroup(slotSharingGroup()) + .setParallelism(parallelism()); + + // Ignore the file deletion result and return the DataStream directly + return result; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java similarity index 98% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java index 085fbfecd270..9f4fb069aae8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/JdbcLockFactory.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import java.io.IOException; import java.sql.DatabaseMetaData; @@ -38,10 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * JDBC table backed implementation of the {@link - * org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory}. - */ +/** JDBC table backed implementation of the {@link TriggerLockFactory}. */ public class JdbcLockFactory implements TriggerLockFactory { private static final Logger LOG = LoggerFactory.getLogger(JdbcLockFactory.class); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java new file mode 100644 index 000000000000..3fc431d02547 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskBuilder.java @@ -0,0 +1,227 @@ +/* + * 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.flink.maintenance.api; + +import java.time.Duration; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.TriggerEvaluator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Experimental +@SuppressWarnings("unchecked") +public abstract class MaintenanceTaskBuilder> { + private int index; + private String taskName; + private String tableName; + private TableLoader tableLoader; + private String uidSuffix = null; + private String slotSharingGroup = null; + private Integer parallelism = null; + private final TriggerEvaluator.Builder triggerEvaluator = new TriggerEvaluator.Builder(); + + abstract DataStream append(DataStream sourceStream); + + /** + * After a given number of Iceberg table commits since the last run, starts the downstream job. + * + * @param commitCount after the downstream job should be started + */ + public T scheduleOnCommitCount(int commitCount) { + triggerEvaluator.commitCount(commitCount); + return (T) this; + } + + /** + * After a given number of new data files since the last run, starts the downstream job. + * + * @param dataFileCount after the downstream job should be started + */ + public T scheduleOnDataFileCount(int dataFileCount) { + triggerEvaluator.dataFileCount(dataFileCount); + return (T) this; + } + + /** + * After a given aggregated data file size since the last run, starts the downstream job. + * + * @param dataFileSizeInBytes after the downstream job should be started + */ + public T scheduleOnDataFileSize(long dataFileSizeInBytes) { + triggerEvaluator.dataFileSizeInBytes(dataFileSizeInBytes); + return (T) this; + } + + /** + * After a given number of new positional delete files since the last run, starts the downstream + * job. + * + * @param posDeleteFileCount after the downstream job should be started + */ + public T scheduleOnPosDeleteFileCount(int posDeleteFileCount) { + triggerEvaluator.posDeleteFileCount(posDeleteFileCount); + return (T) this; + } + + /** + * After a given number of new positional delete records since the last run, starts the downstream + * job. + * + * @param posDeleteRecordCount after the downstream job should be started + */ + public T scheduleOnPosDeleteRecordCount(long posDeleteRecordCount) { + triggerEvaluator.posDeleteRecordCount(posDeleteRecordCount); + return (T) this; + } + + /** + * After a given number of new equality delete files since the last run, starts the downstream + * job. + * + * @param eqDeleteFileCount after the downstream job should be started + */ + public T scheduleOnEqDeleteFileCount(int eqDeleteFileCount) { + triggerEvaluator.eqDeleteFileCount(eqDeleteFileCount); + return (T) this; + } + + /** + * After a given number of new equality delete records since the last run, starts the downstream + * job. + * + * @param eqDeleteRecordCount after the downstream job should be started + */ + public T scheduleOnEqDeleteRecordCount(long eqDeleteRecordCount) { + triggerEvaluator.eqDeleteRecordCount(eqDeleteRecordCount); + return (T) this; + } + + /** + * After a given time since the last run, starts the downstream job. + * + * @param interval after the downstream job should be started + */ + public T scheduleOnInterval(Duration interval) { + triggerEvaluator.timeout(interval); + return (T) this; + } + + /** + * The suffix used for the generated {@link org.apache.flink.api.dag.Transformation}'s uid. + * + * @param newUidSuffix for the transformations + */ + public T uidSuffix(String newUidSuffix) { + this.uidSuffix = newUidSuffix; + return (T) this; + } + + /** + * The {@link SingleOutputStreamOperator#slotSharingGroup(String)} for all the operators of the + * generated stream. Could be used to separate the resources used by this task. + * + * @param newSlotSharingGroup to be used for the operators + */ + public T slotSharingGroup(String newSlotSharingGroup) { + this.slotSharingGroup = newSlotSharingGroup; + return (T) this; + } + + /** + * Sets the parallelism for the stream. + * + * @param newParallelism the required parallelism + */ + public T parallelism(int newParallelism) { + OperatorValidationUtils.validateParallelism(newParallelism); + this.parallelism = newParallelism; + return (T) this; + } + + protected int index() { + return index; + } + + protected String taskName() { + return taskName; + } + + protected String tableName() { + return tableName; + } + + protected TableLoader tableLoader() { + return tableLoader; + } + + protected String uidSuffix() { + return uidSuffix; + } + + protected String slotSharingGroup() { + return slotSharingGroup; + } + + protected Integer parallelism() { + return parallelism; + } + + protected String operatorName(String operatorNameBase) { + return operatorNameBase + "[" + index() + "]"; + } + + TriggerEvaluator evaluator() { + return triggerEvaluator.build(); + } + + DataStream append( + DataStream sourceStream, + int taskIndex, + String newTaskName, + String newTableName, + TableLoader newTableLoader, + String defaultUidSuffix, + String defaultSlotSharingGroup, + int defaultParallelism) { + Preconditions.checkNotNull(newTaskName, "Task name should not be null"); + Preconditions.checkNotNull(newTableLoader, "TableLoader should not be null"); + + this.index = taskIndex; + this.taskName = newTaskName; + this.tableName = newTableName; + this.tableLoader = newTableLoader; + + if (uidSuffix == null) { + uidSuffix = this.taskName + "_" + index + "_" + defaultUidSuffix; + } + + if (parallelism == null) { + parallelism = defaultParallelism; + } + + if (slotSharingGroup == null) { + slotSharingGroup = defaultSlotSharingGroup; + } + + return append(sourceStream); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java new file mode 100644 index 000000000000..f931a9127137 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TableMaintenance.java @@ -0,0 +1,330 @@ +/* + * 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.flink.maintenance.api; + +import java.io.IOException; +import java.time.Duration; +import java.util.List; +import java.util.UUID; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.eventtime.TimestampAssigner; +import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; +import org.apache.flink.api.common.eventtime.WatermarkOutput; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.LockRemover; +import org.apache.iceberg.flink.maintenance.operator.MonitorSource; +import org.apache.iceberg.flink.maintenance.operator.TableChange; +import org.apache.iceberg.flink.maintenance.operator.TriggerEvaluator; +import org.apache.iceberg.flink.maintenance.operator.TriggerManager; +import org.apache.iceberg.flink.sink.IcebergSink; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Creates the table maintenance graph. */ +public class TableMaintenance { + static final String SOURCE_OPERATOR_NAME_PREFIX = "Monitor source for "; + static final String TRIGGER_MANAGER_OPERATOR_NAME = "Trigger manager"; + static final String WATERMARK_ASSIGNER_OPERATOR_NAME = "Watermark Assigner"; + static final String FILTER_OPERATOR_NAME_PREFIX = "Filter "; + static final String LOCK_REMOVER_OPERATOR_NAME = "Lock remover"; + + private TableMaintenance() {} + + /** + * Use when the change stream is already provided, like in the {@link + * IcebergSink#addPostCommitTopology(DataStream)}. + * + * @param changeStream the table changes + * @param tableLoader used for accessing the table + * @param lockFactory used for preventing concurrent task runs + * @return builder for the maintenance stream + */ + @Internal + public static Builder forChangeStream( + DataStream changeStream, + TableLoader tableLoader, + TriggerLockFactory lockFactory) { + Preconditions.checkNotNull(changeStream, "The change stream should not be null"); + Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); + Preconditions.checkNotNull(lockFactory, "LockFactory should not be null"); + + return new Builder(null, changeStream, tableLoader, lockFactory); + } + + /** + * Use this for standalone maintenance job. It creates a monitor source that detect table changes + * and build the maintenance pipelines afterwards. + * + * @param env used to register the monitor source + * @param tableLoader used for accessing the table + * @param lockFactory used for preventing concurrent task runs + * @return builder for the maintenance stream + */ + public static Builder forTable( + StreamExecutionEnvironment env, TableLoader tableLoader, TriggerLockFactory lockFactory) { + Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); + Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); + Preconditions.checkNotNull(lockFactory, "LockFactory should not be null"); + + return new Builder(env, null, tableLoader, lockFactory); + } + + public static class Builder { + private final StreamExecutionEnvironment env; + private final DataStream inputStream; + private final TableLoader tableLoader; + private final List> taskBuilders; + private final TriggerLockFactory lockFactory; + + private String uidSuffix = "TableMaintenance-" + UUID.randomUUID(); + private String slotSharingGroup = StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP; + private Duration rateLimit = Duration.ofMinutes(1); + private Duration lockCheckDelay = Duration.ofSeconds(30); + private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT; + private int maxReadBack = 100; + + private Builder( + StreamExecutionEnvironment env, + DataStream inputStream, + TableLoader tableLoader, + TriggerLockFactory lockFactory) { + this.env = env; + this.inputStream = inputStream; + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.taskBuilders = Lists.newArrayListWithCapacity(4); + } + + /** + * The suffix used for the generated {@link Transformation}'s uid. + * + * @param newUidSuffix for the transformations + */ + public Builder uidSuffix(String newUidSuffix) { + this.uidSuffix = newUidSuffix; + return this; + } + + /** + * The {@link SingleOutputStreamOperator#slotSharingGroup(String)} for all the operators of the + * generated stream. Could be used to separate the resources used by this task. + * + * @param newSlotSharingGroup to be used for the operators + */ + public Builder slotSharingGroup(String newSlotSharingGroup) { + this.slotSharingGroup = newSlotSharingGroup; + return this; + } + + /** + * Limits the firing frequency for the task triggers. + * + * @param newRateLimit firing frequency + */ + public Builder rateLimit(Duration newRateLimit) { + Preconditions.checkNotNull(rateLimit.toMillis() > 0, "Rate limit should be greater than 0"); + this.rateLimit = newRateLimit; + return this; + } + + /** + * Sets the delay for checking lock availability when a concurrent run is detected. + * + * @param newLockCheckDelay lock checking frequency + */ + public Builder lockCheckDelay(Duration newLockCheckDelay) { + this.lockCheckDelay = newLockCheckDelay; + return this; + } + + /** + * Sets the default parallelism of maintenance tasks. Could be overwritten by the {@link + * MaintenanceTaskBuilder#parallelism(int)}. + * + * @param newParallelism task parallelism + */ + public Builder parallelism(int newParallelism) { + OperatorValidationUtils.validateParallelism(newParallelism); + this.parallelism = newParallelism; + return this; + } + + /** + * Maximum number of snapshots checked when started with an embedded {@link MonitorSource} at + * the first time. Only available when the {@link + * TableMaintenance#forTable(StreamExecutionEnvironment, TableLoader, TriggerLockFactory)} is + * used. + * + * @param newMaxReadBack snapshots to consider when initializing + */ + public Builder maxReadBack(int newMaxReadBack) { + Preconditions.checkArgument( + inputStream == null, "Can't set maxReadBack when change stream is provided"); + this.maxReadBack = newMaxReadBack; + return this; + } + + /** + * Adds a specific task with the given schedule. + * + * @param task to add + */ + public Builder add(MaintenanceTaskBuilder task) { + taskBuilders.add(task); + return this; + } + + /** Builds the task graph for the maintenance tasks. */ + public void append() throws IOException { + Preconditions.checkArgument(!taskBuilders.isEmpty(), "Provide at least one task"); + Preconditions.checkNotNull(uidSuffix, "Uid suffix should no be null"); + + List taskNames = Lists.newArrayListWithCapacity(taskBuilders.size()); + List evaluators = Lists.newArrayListWithCapacity(taskBuilders.size()); + for (int i = 0; i < taskBuilders.size(); ++i) { + taskNames.add(nameFor(taskBuilders.get(i), i)); + evaluators.add(taskBuilders.get(i).evaluator()); + } + + try (TableLoader loader = tableLoader.clone()) { + loader.open(); + String tableName = loader.loadTable().name(); + DataStream triggers = + DataStreamUtils.reinterpretAsKeyedStream( + changeStream(tableName, loader), unused -> true) + .process( + new TriggerManager( + loader, + lockFactory, + taskNames, + evaluators, + rateLimit.toMillis(), + lockCheckDelay.toMillis())) + .name(TRIGGER_MANAGER_OPERATOR_NAME) + .uid(TRIGGER_MANAGER_OPERATOR_NAME + uidSuffix) + .slotSharingGroup(slotSharingGroup) + .forceNonParallel() + .assignTimestampsAndWatermarks(new PunctuatedWatermarkStrategy()) + .name(WATERMARK_ASSIGNER_OPERATOR_NAME) + .uid(WATERMARK_ASSIGNER_OPERATOR_NAME + uidSuffix) + .slotSharingGroup(slotSharingGroup) + .forceNonParallel(); + + // Add the specific tasks + DataStream unioned = null; + for (int i = 0; i < taskBuilders.size(); ++i) { + int taskIndex = i; + DataStream filtered = + triggers + .filter(t -> t.taskId() != null && t.taskId() == taskIndex) + .name(FILTER_OPERATOR_NAME_PREFIX + taskIndex) + .forceNonParallel() + .uid(FILTER_OPERATOR_NAME_PREFIX + taskIndex + "-" + uidSuffix) + .slotSharingGroup(slotSharingGroup); + MaintenanceTaskBuilder builder = taskBuilders.get(taskIndex); + DataStream result = + builder.append( + filtered, + taskIndex, + taskNames.get(taskIndex), + tableName, + loader, + uidSuffix, + slotSharingGroup, + parallelism); + if (unioned == null) { + unioned = result; + } else { + unioned = unioned.union(result); + } + } + + // Add the LockRemover to the end + unioned + .transform( + LOCK_REMOVER_OPERATOR_NAME, + TypeInformation.of(Void.class), + new LockRemover(tableName, lockFactory, taskNames)) + .forceNonParallel() + .uid("lock-remover-" + uidSuffix) + .slotSharingGroup(slotSharingGroup); + } + } + + private DataStream changeStream(String tableName, TableLoader loader) { + if (inputStream == null) { + // Create a monitor source to provide the TableChange stream + MonitorSource source = + new MonitorSource( + loader, RateLimiterStrategy.perSecond(1.0 / rateLimit.getSeconds()), maxReadBack); + return env.fromSource( + source, WatermarkStrategy.noWatermarks(), SOURCE_OPERATOR_NAME_PREFIX + tableName) + .uid(SOURCE_OPERATOR_NAME_PREFIX + uidSuffix) + .slotSharingGroup(slotSharingGroup) + .forceNonParallel(); + } else { + return inputStream.global(); + } + } + + private static String nameFor(MaintenanceTaskBuilder streamBuilder, int taskIndex) { + return String.format( + "%s [%s]", streamBuilder.getClass().getSimpleName(), String.valueOf(taskIndex)); + } + } + + @Internal + public static class PunctuatedWatermarkStrategy implements WatermarkStrategy { + @Override + public WatermarkGenerator createWatermarkGenerator( + WatermarkGeneratorSupplier.Context context) { + return new WatermarkGenerator<>() { + @Override + public void onEvent(Trigger event, long eventTimestamp, WatermarkOutput output) { + output.emitWatermark(new Watermark(event.timestamp())); + } + + @Override + public void onPeriodicEmit(WatermarkOutput output) { + // No periodic watermarks + } + }; + } + + @Override + public TimestampAssigner createTimestampAssigner( + TimestampAssignerSupplier.Context context) { + return (element, unused) -> element.timestamp(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java similarity index 93% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java index 06f10f1c1d68..ca1462526f13 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TaskResult.java @@ -16,15 +16,14 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; +import java.io.Serializable; import java.util.List; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; /** The result of a single Maintenance Task. */ -@Internal -public class TaskResult { +public class TaskResult implements Serializable { private final int taskIndex; private final long startEpoch; private final boolean success; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java similarity index 67% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java index 85c6c8dbdd55..09209ba15153 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/Trigger.java @@ -16,47 +16,41 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import org.apache.flink.annotation.Internal; -import org.apache.iceberg.SerializableTable; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -@Internal -class Trigger { +public class Trigger { private final long timestamp; - private final SerializableTable table; private final Integer taskId; private final boolean isRecovery; - private Trigger(long timestamp, SerializableTable table, Integer taskId, boolean isRecovery) { + private Trigger(long timestamp, Integer taskId, boolean isRecovery) { this.timestamp = timestamp; - this.table = table; this.taskId = taskId; this.isRecovery = isRecovery; } - static Trigger create(long timestamp, SerializableTable table, int taskId) { - return new Trigger(timestamp, table, taskId, false); + @Internal + public static Trigger create(long timestamp, int taskId) { + return new Trigger(timestamp, taskId, false); } - static Trigger recovery(long timestamp) { - return new Trigger(timestamp, null, null, true); + @Internal + public static Trigger recovery(long timestamp) { + return new Trigger(timestamp, null, true); } - long timestamp() { + public long timestamp() { return timestamp; } - SerializableTable table() { - return table; - } - - Integer taskId() { + public Integer taskId() { return taskId; } - boolean isRecovery() { + public boolean isRecovery() { return isRecovery; } @@ -64,7 +58,6 @@ boolean isRecovery() { public String toString() { return MoreObjects.toStringHelper(this) .add("timestamp", timestamp) - .add("table", table == null ? null : table.name()) .add("taskId", taskId) .add("isRecovery", isRecovery) .toString(); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java similarity index 86% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java index 329223d27ccf..c31381355efe 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/TriggerLockFactory.java @@ -16,11 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import java.io.Closeable; import java.io.Serializable; import org.apache.flink.annotation.Experimental; +import org.apache.iceberg.flink.maintenance.operator.LockRemover; +import org.apache.iceberg.flink.maintenance.operator.TriggerManager; /** Lock interface for handling locks for the Flink Table Maintenance jobs. */ @Experimental @@ -51,12 +53,11 @@ interface Lock { */ boolean isHeld(); - // TODO: Fix the link to the LockRemover when we have a final name and implementation /** * Releases the lock. Should not fail if the lock is not held by anyone. * - *

    Called by LockRemover. Implementations could assume that are no concurrent calls for this - * method. + *

    Called by {@link LockRemover}. Implementations could assume that are no concurrent calls + * for this method. */ void unlock(); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java new file mode 100644 index 000000000000..dc7846c4c4d3 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java @@ -0,0 +1,130 @@ +/* + * 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.flink.maintenance.operator; + +import java.util.Set; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.BulkDeletionFailureException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.SupportsBulkOperations; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Delete the files using the {@link FileIO} which implements {@link SupportsBulkOperations}. */ +@Internal +public class DeleteFilesProcessor extends AbstractStreamOperator + implements OneInputStreamOperator { + private static final Logger LOG = LoggerFactory.getLogger(DeleteFilesProcessor.class); + + private final String taskIndex; + private final String taskName; + private final SupportsBulkOperations io; + private final String tableName; + private final Set filesToDelete = Sets.newHashSet(); + private final int batchSize; + + private transient Counter failedCounter; + private transient Counter succeededCounter; + + public DeleteFilesProcessor(int taskIndex, String taskName, Table table, int batchSize) { + Preconditions.checkNotNull(taskName, "Task name should no be null"); + Preconditions.checkNotNull(table, "Table should no be null"); + + FileIO fileIO = table.io(); + Preconditions.checkArgument( + fileIO instanceof SupportsBulkOperations, + "%s doesn't support bulk delete", + fileIO.getClass().getSimpleName()); + + this.taskIndex = String.valueOf(taskIndex); + this.taskName = taskName; + this.io = (SupportsBulkOperations) fileIO; + this.tableName = table.name(); + this.batchSize = batchSize; + } + + @Override + public void open() throws Exception { + this.failedCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, taskName) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, taskIndex) + .counter(TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER); + this.succeededCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, taskName) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, taskIndex) + .counter(TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + if (element.isRecord()) { + filesToDelete.add(element.getValue()); + } + + if (filesToDelete.size() >= batchSize) { + deleteFiles(); + } + } + + @Override + public void processWatermark(Watermark mark) { + deleteFiles(); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) { + deleteFiles(); + } + + private void deleteFiles() { + try { + io.deleteFiles(filesToDelete); + LOG.info( + "Deleted {} files from table {} using bulk deletes", filesToDelete.size(), tableName); + succeededCounter.inc(filesToDelete.size()); + filesToDelete.clear(); + } catch (BulkDeletionFailureException e) { + int deletedFilesCount = filesToDelete.size() - e.numberFailedObjects(); + LOG.warn( + "Deleted only {} of {} files from table {} using bulk deletes", + deletedFilesCount, + filesToDelete.size(), + tableName, + e); + succeededCounter.inc(deletedFilesCount); + failedCounter.inc(e.numberFailedObjects()); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java new file mode 100644 index 000000000000..a09d0244e95d --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -0,0 +1,129 @@ +/* + * 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.flink.maintenance.operator; + +import java.util.Collections; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; +import org.apache.iceberg.ExpireSnapshots; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Calls the {@link ExpireSnapshots} to remove the old snapshots and emits the filenames which could + * be removed in the {@link #DELETE_STREAM} side output. + */ +@Internal +public class ExpireSnapshotsProcessor extends ProcessFunction { + private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsProcessor.class); + public static final OutputTag DELETE_STREAM = + new OutputTag<>("expire-snapshots-file-deletes-stream", Types.STRING); + + private final TableLoader tableLoader; + private final Long maxSnapshotAgeMs; + private final Integer numSnapshots; + private final Integer plannerPoolSize; + private transient ExecutorService plannerPool; + private transient Table table; + + public ExpireSnapshotsProcessor( + TableLoader tableLoader, + Long maxSnapshotAgeMs, + Integer numSnapshots, + Integer plannerPoolSize) { + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + + this.tableLoader = tableLoader; + this.maxSnapshotAgeMs = maxSnapshotAgeMs; + this.numSnapshots = numSnapshots; + this.plannerPoolSize = plannerPoolSize; + } + + @Override + public void open(Configuration parameters) throws Exception { + tableLoader.open(); + this.table = tableLoader.loadTable(); + this.plannerPool = + plannerPoolSize != null + ? ThreadPools.newWorkerPool(table.name() + "-table--planner", plannerPoolSize) + : ThreadPools.getWorkerPool(); + } + + @Override + public void processElement(Trigger trigger, Context ctx, Collector out) + throws Exception { + try { + table.refresh(); + ExpireSnapshots expireSnapshots = table.expireSnapshots(); + if (maxSnapshotAgeMs != null) { + expireSnapshots = expireSnapshots.expireOlderThan(ctx.timestamp() - maxSnapshotAgeMs); + } + + if (numSnapshots != null) { + expireSnapshots = expireSnapshots.retainLast(numSnapshots); + } + + AtomicLong deleteFileCounter = new AtomicLong(0L); + expireSnapshots + .planWith(plannerPool) + .deleteWith( + file -> { + ctx.output(DELETE_STREAM, file); + deleteFileCounter.incrementAndGet(); + }) + .cleanExpiredFiles(true) + .commit(); + + LOG.info( + "Successfully finished expiring snapshots for {} at {}. Scheduled {} files for delete.", + table, + ctx.timestamp(), + deleteFileCounter.get()); + out.collect( + new TaskResult(trigger.taskId(), trigger.timestamp(), true, Collections.emptyList())); + } catch (Exception e) { + LOG.error("Failed to expiring snapshots for {} at {}", table, ctx.timestamp(), e); + out.collect( + new TaskResult(trigger.taskId(), trigger.timestamp(), false, Lists.newArrayList(e))); + } + } + + @Override + public void close() throws Exception { + super.close(); + + tableLoader.close(); + if (plannerPoolSize != null) { + plannerPool.shutdown(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java index 3c3761ef2f4d..14d590162c8b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java @@ -26,6 +26,9 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; @@ -64,6 +67,7 @@ public class LockRemover extends AbstractStreamOperator implements OneInputStreamOperator { private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + private final String tableName; private final TriggerLockFactory lockFactory; private final List maintenanceTaskNames; @@ -74,12 +78,14 @@ public class LockRemover extends AbstractStreamOperator private transient TriggerLockFactory.Lock recoveryLock; private transient long lastProcessedTaskStartEpoch = 0L; - public LockRemover(TriggerLockFactory lockFactory, List maintenanceTaskNames) { + public LockRemover( + String tableName, TriggerLockFactory lockFactory, List maintenanceTaskNames) { Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); Preconditions.checkArgument( maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(), "Invalid maintenance task names: null or empty"); + this.tableName = tableName; this.lockFactory = lockFactory; this.maintenanceTaskNames = maintenanceTaskNames; } @@ -91,22 +97,31 @@ public void open() throws Exception { Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); this.failedTaskResultCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); this.taskLastRunDurationMs = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); - for (String name : maintenanceTaskNames) { + for (int taskIndex = 0; taskIndex < maintenanceTaskNames.size(); ++taskIndex) { succeededTaskResultCounters.add( getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) .counter(TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER)); failedTaskResultCounters.add( getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) .counter(TableMaintenanceMetrics.FAILED_TASK_COUNTER)); AtomicLong duration = new AtomicLong(0); taskLastRunDurationMs.add(duration); getRuntimeContext() .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) .gauge(TableMaintenanceMetrics.LAST_RUN_DURATION_MS, duration::get); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java index 89efffa15f16..d74b2349b1de 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java @@ -43,7 +43,7 @@ /** Monitors an Iceberg table for changes */ @Internal -class MonitorSource extends SingleThreadedIteratorSource { +public class MonitorSource extends SingleThreadedIteratorSource { private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final TableLoader tableLoader; @@ -58,7 +58,7 @@ class MonitorSource extends SingleThreadedIteratorSource { * @param rateLimiterStrategy limits the frequency the table is checked * @param maxReadBack sets the number of snapshots read before stopping change collection */ - MonitorSource( + public MonitorSource( TableLoader tableLoader, RateLimiterStrategy rateLimiterStrategy, long maxReadBack) { Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); Preconditions.checkNotNull(rateLimiterStrategy, "Rate limiter strategy should no be null"); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java index 773b34b6c495..78be0a9c15d6 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -28,7 +28,7 @@ /** Event describing changes in an Iceberg table */ @Internal -class TableChange { +public class TableChange { private int dataFileCount; private long dataFileSizeInBytes; private int posDeleteFileCount; @@ -37,7 +37,7 @@ class TableChange { private long eqDeleteRecordCount; private int commitCount; - TableChange( + private TableChange( int dataFileCount, long dataFileSizeInBytes, int posDeleteFileCount, @@ -87,7 +87,7 @@ static TableChange empty() { return new TableChange(0, 0L, 0, 0L, 0, 0L, 0); } - static Builder builder() { + public static Builder builder() { return new Builder(); } @@ -115,7 +115,7 @@ long eqDeleteRecordCount() { return eqDeleteRecordCount; } - public int commitCount() { + int commitCount() { return commitCount; } @@ -183,7 +183,7 @@ public int hashCode() { commitCount); } - static class Builder { + public static class Builder { private int dataFileCount = 0; private long dataFileSizeInBytes = 0L; private int posDeleteFileCount = 0; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java index 1a04461aed43..6147c3a5fd16 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java @@ -19,8 +19,10 @@ package org.apache.iceberg.flink.maintenance.operator; public class TableMaintenanceMetrics { - public static final String GROUP_KEY = "maintenanceTask"; - public static final String GROUP_VALUE_DEFAULT = "maintenanceTask"; + public static final String GROUP_KEY = "maintenance"; + public static final String TASK_NAME_KEY = "taskName"; + public static final String TASK_INDEX_KEY = "taskIndex"; + public static final String TABLE_NAME_KEY = "tableName"; // TriggerManager metrics public static final String RATE_LIMITER_TRIGGERED = "rateLimiterTriggered"; @@ -33,6 +35,10 @@ public class TableMaintenanceMetrics { public static final String FAILED_TASK_COUNTER = "failedTasks"; public static final String LAST_RUN_DURATION_MS = "lastRunDurationMs"; + // DeleteFiles metrics + public static final String DELETE_FILE_FAILED_COUNTER = "deleteFailed"; + public static final String DELETE_FILE_SUCCEEDED_COUNTER = "deleteSucceeded"; + private TableMaintenanceMetrics() { // do not instantiate } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java index dba33b22a42a..d448898bdfe6 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java @@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory; @Internal -class TriggerEvaluator implements Serializable { +public class TriggerEvaluator implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(TriggerEvaluator.class); private final List predicates; @@ -50,7 +50,7 @@ boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { return result; } - static class Builder implements Serializable { + public static class Builder implements Serializable { private Integer dataFileCount; private Long dataFileSizeInBytes; private Integer posDeleteFileCount; @@ -95,12 +95,12 @@ public Builder commitCount(int newCommitCount) { return this; } - Builder timeout(Duration newTimeout) { + public Builder timeout(Duration newTimeout) { this.timeout = newTimeout; return this; } - TriggerEvaluator build() { + public TriggerEvaluator build() { List predicates = Lists.newArrayList(); if (dataFileCount != null) { predicates.add((change, unused, unused2) -> change.dataFileCount() >= dataFileCount); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java index dc95b27af0a6..a96e99d94299 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.List; -import java.util.stream.Collectors; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -36,8 +35,9 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.util.Collector; -import org.apache.iceberg.SerializableTable; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; @@ -57,11 +57,11 @@ * the timer functions are available, but the key is not used. */ @Internal -class TriggerManager extends KeyedProcessFunction +public class TriggerManager extends KeyedProcessFunction implements CheckpointedFunction { private static final Logger LOG = LoggerFactory.getLogger(TriggerManager.class); - private final TableLoader tableLoader; + private final String tableName; private final TriggerLockFactory lockFactory; private final List maintenanceTaskNames; private final List evaluators; @@ -89,7 +89,7 @@ class TriggerManager extends KeyedProcessFunction private transient int startsFrom = 0; private transient boolean triggered = false; - TriggerManager( + public TriggerManager( TableLoader tableLoader, TriggerLockFactory lockFactory, List maintenanceTaskNames, @@ -110,7 +110,8 @@ class TriggerManager extends KeyedProcessFunction Preconditions.checkArgument( lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); - this.tableLoader = tableLoader; + tableLoader.open(); + this.tableName = tableLoader.loadTable().name(); this.lockFactory = lockFactory; this.maintenanceTaskNames = maintenanceTaskNames; this.evaluators = evaluators; @@ -123,30 +124,32 @@ public void open(Configuration parameters) throws Exception { this.rateLimiterTriggeredCounter = getRuntimeContext() .getMetricGroup() - .addGroup( - TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); this.concurrentRunThrottledCounter = getRuntimeContext() .getMetricGroup() - .addGroup( - TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) .counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED); this.nothingToTriggerCounter = getRuntimeContext() .getMetricGroup() - .addGroup( - TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); - this.triggerCounters = - maintenanceTaskNames.stream() - .map( - name -> - getRuntimeContext() - .getMetricGroup() - .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) - .counter(TableMaintenanceMetrics.TRIGGERED)) - .collect(Collectors.toList()); + this.triggerCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + for (int taskIndex = 0; taskIndex < maintenanceTaskNames.size(); ++taskIndex) { + triggerCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY) + .addGroup(TableMaintenanceMetrics.TABLE_NAME_KEY, tableName) + .addGroup(TableMaintenanceMetrics.TASK_NAME_KEY, maintenanceTaskNames.get(taskIndex)) + .addGroup(TableMaintenanceMetrics.TASK_INDEX_KEY, String.valueOf(taskIndex)) + .counter(TableMaintenanceMetrics.TRIGGERED)); + } this.nextEvaluationTimeState = getRuntimeContext() @@ -159,8 +162,6 @@ public void open(Configuration parameters) throws Exception { this.lastTriggerTimesState = getRuntimeContext() .getListState(new ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG)); - - tableLoader.open(); } @Override @@ -220,7 +221,6 @@ public void onTimer(long timestamp, OnTimerContext ctx, Collector out) @Override public void close() throws IOException { - tableLoader.close(); lockFactory.close(); } @@ -256,10 +256,8 @@ private void checkAndFire(long current, TimerService timerService, Collector + *

  • {@link StreamExecutionEnvironment} - environment for testing + *
  • {@link ManualSource} - source for manually emitting {@link Trigger}s + *
  • {@link DataStream} - which generated from the {@link ManualSource} + *
  • {@link CollectingSink} - which could be used poll for the records emitted by the + * maintenance tasks + * + */ +class MaintenanceTaskInfraExtension implements BeforeEachCallback { + private StreamExecutionEnvironment env; + private ManualSource source; + private DataStream triggerStream; + private CollectingSink sink; + + @Override + public void beforeEach(ExtensionContext context) { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + source = new ManualSource<>(env, TypeInformation.of(Trigger.class)); + // Adds the watermark to mimic the behaviour expected for the input of the maintenance tasks + triggerStream = + source + .dataStream() + .assignTimestampsAndWatermarks(new TableMaintenance.PunctuatedWatermarkStrategy()) + .name(IGNORED_OPERATOR_NAME) + .forceNonParallel(); + sink = new CollectingSink<>(); + } + + StreamExecutionEnvironment env() { + return env; + } + + ManualSource source() { + return source; + } + + DataStream triggerStream() { + return triggerStream; + } + + CollectingSink sink() { + return sink; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java new file mode 100644 index 000000000000..36041d9c3831 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/MaintenanceTaskTestBase.java @@ -0,0 +1,64 @@ +/* + * 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.flink.maintenance.api; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.function.Supplier; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.flink.maintenance.operator.CollectingSink; +import org.apache.iceberg.flink.maintenance.operator.ManualSource; +import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.extension.RegisterExtension; + +class MaintenanceTaskTestBase extends OperatorTestBase { + private static final int TESTING_TASK_ID = 0; + private static final Duration POLL_DURATION = Duration.ofSeconds(5); + + @RegisterExtension MaintenanceTaskInfraExtension infra = new MaintenanceTaskInfraExtension(); + + void runAndWaitForSuccess( + StreamExecutionEnvironment env, + ManualSource triggerSource, + CollectingSink collectingSink, + Supplier waitForCondition) + throws Exception { + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Do a single task run + long time = System.currentTimeMillis(); + triggerSource.sendRecord(Trigger.create(time, TESTING_TASK_ID), time); + + TaskResult result = collectingSink.poll(POLL_DURATION); + + assertThat(result.startEpoch()).isEqualTo(time); + assertThat(result.success()).isTrue(); + assertThat(result.taskIndex()).isEqualTo(TESTING_TASK_ID); + + Awaitility.await().until(waitForCondition::get); + } finally { + closeJobClient(jobClient); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java new file mode 100644 index 000000000000..f80129f966e1 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java @@ -0,0 +1,254 @@ +/* + * 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.flink.maintenance.api; + +import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; +import static org.apache.iceberg.flink.maintenance.api.ExpireSnapshots.DELETE_FILES_OPERATOR_NAME; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.List; +import java.util.Set; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestExpireSnapshots extends MaintenanceTaskTestBase { + private Table table; + + @BeforeEach + void before() { + MetricsReporterFactoryForTests.reset(); + this.table = createTable(); + tableLoader().open(); + } + + @Test + void testExpireSnapshots() throws Exception { + insert(table, 1, "a"); + insert(table, 2, "b"); + insert(table, 3, "c"); + insert(table, 4, "d"); + + Set snapshots = Sets.newHashSet(table.snapshots()); + assertThat(snapshots).hasSize(4); + + ExpireSnapshots.builder() + .parallelism(1) + .planningWorkerPoolSize(2) + .deleteBatchSize(3) + .maxSnapshotAge(Duration.ZERO) + .retainLast(1) + .uidSuffix(UID_SUFFIX) + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + "OTHER", + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + runAndWaitForSuccess( + infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(table.name(), 3L)); + + table.refresh(); + assertThat(Sets.newHashSet(table.snapshots())).hasSize(1); + // Check that the table data not changed + SimpleDataUtil.assertTableRecords( + table, + ImmutableList.of( + createRecord(1, "a"), + createRecord(2, "b"), + createRecord(3, "c"), + createRecord(4, "d"))); + } + + @Test + void testFailure() throws Exception { + insert(table, 1, "a"); + insert(table, 2, "b"); + + ExpireSnapshots.builder() + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + UID_SUFFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + JobClient jobClient = null; + try { + jobClient = infra.env().executeAsync(); + + // Do a single task run + long time = System.currentTimeMillis(); + infra.source().sendRecord(Trigger.create(time, 1), time); + + // First successful run (ensure that the operators are loaded/opened etc.) + assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isTrue(); + + // Drop the table, so it will cause an exception + dropTable(); + + // Failed run + infra.source().sendRecord(Trigger.create(time + 1, 1), time + 1); + + assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isFalse(); + } finally { + closeJobClient(jobClient); + } + + // Check the metrics. There are no expired snapshots or data files because ExpireSnapshots has + // no max age of number of snapshots set, so no files are removed. + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder, Long>() + .put( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_FAILED_COUNTER), + 0L) + .put( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_SUCCEEDED_COUNTER), + 0L) + .build()); + } + + @Test + void testUidAndSlotSharingGroup() { + ExpireSnapshots.builder() + .slotSharingGroup(SLOT_SHARING_GROUP) + .uidSuffix(UID_SUFFIX) + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + UID_SUFFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + checkUidsAreSet(infra.env(), UID_SUFFIX); + checkSlotSharingGroupsAreSet(infra.env(), SLOT_SHARING_GROUP); + } + + @Test + void testUidAndSlotSharingGroupUnset() { + ExpireSnapshots.builder() + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + UID_SUFFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + checkUidsAreSet(infra.env(), null); + checkSlotSharingGroupsAreSet(infra.env(), null); + } + + @Test + void testMetrics() throws Exception { + insert(table, 1, "a"); + insert(table, 2, "b"); + + ExpireSnapshots.builder() + .maxSnapshotAge(Duration.ZERO) + .retainLast(1) + .parallelism(1) + .append( + infra.triggerStream(), + 0, + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + tableLoader(), + UID_SUFFIX, + StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP, + 1) + .sinkTo(infra.sink()); + + runAndWaitForSuccess( + infra.env(), infra.source(), infra.sink(), () -> checkDeleteFinished(table.name(), 1L)); + + // Check the metrics + Awaitility.await() + .untilAsserted( + () -> + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder, Long>() + .put( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_FAILED_COUNTER), + 0L) + .put( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + table.name(), + DUMMY_TASK_NAME, + "0", + DELETE_FILE_SUCCEEDED_COUNTER), + 1L) + .build())); + } + + private static boolean checkDeleteFinished(String tableName, Long expectedDeleteNum) { + return expectedDeleteNum.equals( + MetricsReporterFactoryForTests.counter( + ImmutableList.of( + DELETE_FILES_OPERATOR_NAME + "[0]", + tableName, + DUMMY_TASK_NAME, + "0", + DELETE_FILE_SUCCEEDED_COUNTER))); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java similarity index 92% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java index 051d09d92bad..c8fa2a7d362a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestJdbcLockFactory.java @@ -16,9 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; -import static org.apache.iceberg.flink.maintenance.operator.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; +import static org.apache.iceberg.flink.maintenance.api.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java similarity index 97% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java index bf9e86f2534d..c06bef9087d5 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestLockFactoryBase.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.maintenance.operator; +package org.apache.iceberg.flink.maintenance.api; import static org.assertj.core.api.Assertions.assertThat; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java new file mode 100644 index 000000000000..467ad2d8ced9 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java @@ -0,0 +1,67 @@ +/* + * 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.flink.maintenance.api; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.time.Duration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestMaintenanceE2E extends OperatorTestBase { + private StreamExecutionEnvironment env; + + @BeforeEach + public void beforeEach() throws IOException { + this.env = StreamExecutionEnvironment.getExecutionEnvironment(); + Table table = createTable(); + insert(table, 1, "a"); + } + + @Test + void testE2e() throws Exception { + TableMaintenance.forTable(env, tableLoader(), LOCK_FACTORY) + .uidSuffix("E2eTestUID") + .rateLimit(Duration.ofMinutes(10)) + .lockCheckDelay(Duration.ofSeconds(10)) + .add( + ExpireSnapshots.builder() + .scheduleOnCommitCount(10) + .maxSnapshotAge(Duration.ofMinutes(10)) + .retainLast(5) + .deleteBatchSize(5) + .parallelism(8)) + .append(); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Just make sure that we are able to instantiate the flow + assertThat(jobClient).isNotNull(); + } finally { + closeJobClient(jobClient); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java new file mode 100644 index 000000000000..0e4a72bd16f8 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestTableMaintenance.java @@ -0,0 +1,460 @@ +/* + * 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.flink.maintenance.api; + +import static org.apache.iceberg.flink.SimpleDataUtil.createRowData; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.LOCK_REMOVER_OPERATOR_NAME; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.SOURCE_OPERATOR_NAME_PREFIX; +import static org.apache.iceberg.flink.maintenance.api.TableMaintenance.TRIGGER_MANAGER_OPERATOR_NAME; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.transformations.SourceTransformation; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.operator.ManualSource; +import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests; +import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase; +import org.apache.iceberg.flink.maintenance.operator.TableChange; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +class TestTableMaintenance extends OperatorTestBase { + private static final String[] TASKS = + new String[] { + MaintenanceTaskBuilderForTest.class.getSimpleName() + " [0]", + MaintenanceTaskBuilderForTest.class.getSimpleName() + " [1]" + }; + private static final TableChange DUMMY_CHANGE = TableChange.builder().commitCount(1).build(); + private static final List PROCESSED = + Collections.synchronizedList(Lists.newArrayListWithCapacity(1)); + + private StreamExecutionEnvironment env; + private Table table; + + @TempDir private File checkpointDir; + + @BeforeEach + public void beforeEach() throws IOException { + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + this.env = StreamExecutionEnvironment.getExecutionEnvironment(config); + this.table = createTable(); + insert(table, 1, "a"); + + PROCESSED.clear(); + MaintenanceTaskBuilderForTest.counter = 0; + } + + @Test + void testForChangeStream() throws Exception { + ManualSource schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader(), LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .lockCheckDelay(Duration.ofSeconds(3)) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .scheduleOnDataFileCount(2) + .scheduleOnDataFileSize(3L) + .scheduleOnEqDeleteFileCount(4) + .scheduleOnEqDeleteRecordCount(5L) + .scheduleOnPosDeleteFileCount(6) + .scheduleOnPosDeleteRecordCount(7L) + .scheduleOnInterval(Duration.ofHours(1))); + + sendEvents(schedulerSource, streamBuilder, ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1))); + } + + @Test + void testForTable() throws Exception { + TableLoader tableLoader = tableLoader(); + + env.enableCheckpointing(10); + + TableMaintenance.forTable(env, tableLoader, LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .maxReadBack(2) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(2)) + .append(); + + // Creating a stream for inserting data into the table concurrently + ManualSource insertSource = + new ManualSource<>(env, InternalTypeInfo.of(FlinkSchemaUtil.convert(table.schema()))); + FlinkSink.forRowData(insertSource.dataStream()) + .tableLoader(tableLoader) + .uidPrefix(UID_SUFFIX + "-iceberg-sink") + .append(); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + insertSource.sendRecord(createRowData(2, "b")); + + Awaitility.await().until(() -> PROCESSED.size() == 1); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testLocking() throws Exception { + TriggerLockFactory.Lock lock = LOCK_FACTORY.createLock(); + + ManualSource schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader(), LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)); + + assertThat(lock.isHeld()).isFalse(); + sendEvents(schedulerSource, streamBuilder, ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1))); + + assertThat(lock.isHeld()).isFalse(); + } + + @Test + void testMetrics() throws Exception { + ManualSource schedulerSource = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + + TableMaintenance.Builder streamBuilder = + TableMaintenance.forChangeStream(schedulerSource.dataStream(), tableLoader(), LOCK_FACTORY) + .rateLimit(Duration.ofMillis(2)) + .lockCheckDelay(Duration.ofMillis(2)) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) + .add(new MaintenanceTaskBuilderForTest(false).scheduleOnCommitCount(2)); + + sendEvents( + schedulerSource, + streamBuilder, + ImmutableList.of(Tuple2.of(DUMMY_CHANGE, 1), Tuple2.of(DUMMY_CHANGE, 2))); + + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, + table.name(), + TASKS[0], + "0", + SUCCEEDED_TASK_COUNTER)) + .equals(2L)); + + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder, Long>() + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, + table.name(), + TASKS[0], + "0", + SUCCEEDED_TASK_COUNTER), + 2L) + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, table.name(), TASKS[0], "0", FAILED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), TASKS[0], "0", TRIGGERED), + 2L) + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, + table.name(), + TASKS[1], + "1", + SUCCEEDED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + LOCK_REMOVER_OPERATOR_NAME, table.name(), TASKS[1], "1", FAILED_TASK_COUNTER), + 1L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), TASKS[1], "1", TRIGGERED), + 1L) + .put( + ImmutableList.of(TRIGGER_MANAGER_OPERATOR_NAME, table.name(), NOTHING_TO_TRIGGER), + -1L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), CONCURRENT_RUN_THROTTLED), + -1L) + .put( + ImmutableList.of( + TRIGGER_MANAGER_OPERATOR_NAME, table.name(), RATE_LIMITER_TRIGGERED), + -1L) + .build()); + } + + @Test + void testUidAndSlotSharingGroup() throws IOException { + TableMaintenance.forChangeStream( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader(), + LOCK_FACTORY) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP)) + .append(); + + checkUidsAreSet(env, UID_SUFFIX); + checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); + } + + @Test + void testUidAndSlotSharingGroupUnset() throws IOException { + TableMaintenance.forChangeStream( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader(), + LOCK_FACTORY) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) + .append(); + + checkUidsAreSet(env, null); + checkSlotSharingGroupsAreSet(env, null); + } + + @Test + void testUidAndSlotSharingGroupInherit() throws IOException { + TableMaintenance.forChangeStream( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader(), + LOCK_FACTORY) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add(new MaintenanceTaskBuilderForTest(true).scheduleOnCommitCount(1)) + .append(); + + checkUidsAreSet(env, UID_SUFFIX); + checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); + } + + @Test + void testUidAndSlotSharingGroupOverWrite() throws IOException { + String anotherUid = "Another-UID"; + String anotherSlotSharingGroup = "Another-SlotSharingGroup"; + TableMaintenance.forChangeStream( + new ManualSource<>(env, TypeInformation.of(TableChange.class)).dataStream(), + tableLoader(), + LOCK_FACTORY) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .uidSuffix(anotherUid) + .slotSharingGroup(anotherSlotSharingGroup)) + .append(); + + // Choose an operator from the scheduler part of the graph + Transformation schedulerTransformation = + env.getTransformations().stream() + .filter(t -> t.getName().equals("Trigger manager")) + .findFirst() + .orElseThrow(); + assertThat(schedulerTransformation.getUid()).contains(UID_SUFFIX); + assertThat(schedulerTransformation.getSlotSharingGroup()).isPresent(); + assertThat(schedulerTransformation.getSlotSharingGroup().get().getName()) + .isEqualTo(SLOT_SHARING_GROUP); + + // Choose an operator from the maintenance task part of the graph + Transformation scheduledTransformation = + env.getTransformations().stream() + .filter( + t -> t.getName().startsWith(MaintenanceTaskBuilderForTest.class.getSimpleName())) + .findFirst() + .orElseThrow(); + assertThat(scheduledTransformation.getUid()).contains(anotherUid); + assertThat(scheduledTransformation.getSlotSharingGroup()).isPresent(); + assertThat(scheduledTransformation.getSlotSharingGroup().get().getName()) + .isEqualTo(anotherSlotSharingGroup); + } + + @Test + void testUidAndSlotSharingGroupForMonitorSource() throws IOException { + TableMaintenance.forTable(env, tableLoader(), LOCK_FACTORY) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP) + .add( + new MaintenanceTaskBuilderForTest(true) + .scheduleOnCommitCount(1) + .uidSuffix(UID_SUFFIX) + .slotSharingGroup(SLOT_SHARING_GROUP)) + .append(); + + Transformation source = monitorSource(); + assertThat(source).isNotNull(); + assertThat(source.getUid()).contains(UID_SUFFIX); + assertThat(source.getSlotSharingGroup()).isPresent(); + assertThat(source.getSlotSharingGroup().get().getName()).isEqualTo(SLOT_SHARING_GROUP); + + checkUidsAreSet(env, UID_SUFFIX); + checkSlotSharingGroupsAreSet(env, SLOT_SHARING_GROUP); + } + + /** + * Sends the events though the {@link ManualSource} provided, and waits until the given number of + * records are processed. + * + * @param schedulerSource used for sending the events + * @param streamBuilder used for generating the job + * @param eventsAndResultNumbers the pair of the event and the expected processed records + * @throws Exception if any + */ + private void sendEvents( + ManualSource schedulerSource, + TableMaintenance.Builder streamBuilder, + List> eventsAndResultNumbers) + throws Exception { + streamBuilder.append(); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + eventsAndResultNumbers.forEach( + eventsAndResultNumber -> { + int expectedSize = PROCESSED.size() + eventsAndResultNumber.f1; + schedulerSource.sendRecord(eventsAndResultNumber.f0); + Awaitility.await() + .until( + () -> PROCESSED.size() == expectedSize && !LOCK_FACTORY.createLock().isHeld()); + }); + } finally { + closeJobClient(jobClient); + } + } + + /** + * Finds the {@link org.apache.iceberg.flink.maintenance.operator.MonitorSource} for testing + * purposes by parsing the transformation tree. + * + * @return The monitor source if we found it + */ + private Transformation monitorSource() { + assertThat(env.getTransformations()).isNotEmpty(); + assertThat(env.getTransformations().get(0).getInputs()).isNotEmpty(); + assertThat(env.getTransformations().get(0).getInputs().get(0).getInputs()).isNotEmpty(); + + Transformation result = + env.getTransformations().get(0).getInputs().get(0).getInputs().get(0); + + // Some checks to make sure this is the transformation we are looking for + assertThat(result).isInstanceOf(SourceTransformation.class); + assertThat(result.getName()).startsWith(SOURCE_OPERATOR_NAME_PREFIX); + + return result; + } + + private static class MaintenanceTaskBuilderForTest + extends MaintenanceTaskBuilder { + private final boolean success; + private final int id; + private static int counter = 0; + + MaintenanceTaskBuilderForTest(boolean success) { + this.success = success; + this.id = counter; + ++counter; + } + + @Override + DataStream append(DataStream trigger) { + String name = TASKS[id]; + return trigger + .map(new DummyMaintenanceTask(success)) + .name(name) + .uid(uidSuffix() + "-test-mapper-" + name + "-" + id) + .slotSharingGroup(slotSharingGroup()) + .forceNonParallel(); + } + } + + private static class DummyMaintenanceTask + implements MapFunction, ResultTypeQueryable, Serializable { + private final boolean success; + + private DummyMaintenanceTask(boolean success) { + this.success = success; + } + + @Override + public TaskResult map(Trigger trigger) { + // Ensure that the lock is held when processing + assertThat(LOCK_FACTORY.createLock().isHeld()).isTrue(); + PROCESSED.add(trigger); + + return new TaskResult( + trigger.taskId(), + trigger.timestamp(), + success, + success ? Collections.emptyList() : Lists.newArrayList(new Exception("Testing error"))); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(TaskResult.class); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java index 9b6580fad0bf..e7e818ba6887 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java @@ -31,7 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; /** Sink for collecting output during testing. */ -class CollectingSink implements Sink { +public class CollectingSink implements Sink { private static final long serialVersionUID = 1L; private static final List> QUEUES = Collections.synchronizedList(Lists.newArrayListWithExpectedSize(1)); @@ -39,7 +39,7 @@ class CollectingSink implements Sink { private final int index; /** Creates a new sink which collects the elements received. */ - CollectingSink() { + public CollectingSink() { this.index = NUM_SINKS.incrementAndGet(); QUEUES.add(new LinkedBlockingQueue<>()); } @@ -69,7 +69,7 @@ boolean isEmpty() { * @return The first element received by this {@link Sink} * @throws TimeoutException if no element received until the timeout */ - T poll(Duration timeout) throws TimeoutException { + public T poll(Duration timeout) throws TimeoutException { Object element; try { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java deleted file mode 100644 index 36e162d4f068..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -class ConstantsForTests { - public static final long EVENT_TIME = 10L; - static final long EVENT_TIME_2 = 11L; - static final String DUMMY_NAME = "dummy"; - - private ConstantsForTests() { - // Do not instantiate - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java deleted file mode 100644 index 91d36aa3e85d..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.jupiter.api.extension.AfterEachCallback; -import org.junit.jupiter.api.extension.BeforeEachCallback; -import org.junit.jupiter.api.extension.ExtensionContext; - -/** - * Junit 5 extension for running Flink SQL queries. {@link - * org.apache.flink.test.junit5.MiniClusterExtension} is used for executing the SQL batch jobs. - */ -public class FlinkSqlExtension implements BeforeEachCallback, AfterEachCallback { - private final String catalogName; - private final Map catalogProperties; - private final String databaseName; - private final Path warehouse; - private final CatalogLoader catalogLoader; - private TableEnvironment tableEnvironment; - - public FlinkSqlExtension( - String catalogName, Map catalogProperties, String databaseName) { - this.catalogName = catalogName; - this.catalogProperties = Maps.newHashMap(catalogProperties); - this.databaseName = databaseName; - - // Add temporary dir as a warehouse location - try { - this.warehouse = Files.createTempDirectory("warehouse"); - } catch (IOException e) { - throw new RuntimeException(e); - } - this.catalogProperties.put( - CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouse)); - this.catalogLoader = - CatalogLoader.hadoop(catalogName, new Configuration(), this.catalogProperties); - } - - @Override - public void beforeEach(ExtensionContext context) { - // We need to recreate the tableEnvironment for every test as the minicluster is recreated - this.tableEnvironment = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - exec("CREATE CATALOG %s WITH %s", catalogName, toWithClause(catalogProperties)); - exec("CREATE DATABASE %s.%s", catalogName, databaseName); - exec("USE CATALOG %s", catalogName); - exec("USE %s", databaseName); - } - - @Override - public void afterEach(ExtensionContext context) throws IOException { - List tables = exec("SHOW TABLES"); - tables.forEach(t -> exec("DROP TABLE IF EXISTS %s", t.getField(0))); - exec("USE CATALOG default_catalog"); - exec("DROP CATALOG IF EXISTS %s", catalogName); - try (Stream files = Files.walk(warehouse)) { - files.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); - } - } - - /** - * Executes an SQL query with the given parameters. The parameter substitution is done by {@link - * String#format(String, Object...)}. - * - * @param query to run - * @param parameters to substitute to the query - * @return The {@link Row}s returned by the query - */ - public List exec(String query, Object... parameters) { - TableResult tableResult = tableEnvironment.executeSql(String.format(query, parameters)); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - /** - * Returns the {@link TableLoader} which could be used to access the given table. - * - * @param tableName of the table - * @return the {@link TableLoader} for the table - */ - public TableLoader tableLoader(String tableName) { - TableLoader tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(databaseName, tableName)); - tableLoader.open(); - return tableLoader; - } - - private static String toWithClause(Map props) { - return String.format( - "(%s)", - props.entrySet().stream() - .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) - .collect(Collectors.joining(","))); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java deleted file mode 100644 index 9cdc55cb0cce..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.maintenance.operator; - -import java.io.File; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; -import org.awaitility.Awaitility; - -class FlinkStreamingTestUtils { - private FlinkStreamingTestUtils() { - // Do not instantiate - } - - /** - * Close the {@link JobClient} and wait for the job closure. If the savepointDir is specified, it - * stops the job with a savepoint. - * - * @param jobClient the job to close - * @param savepointDir the savepointDir to store the last savepoint. If null then - * stop without a savepoint. - * @return configuration for restarting the job from the savepoint - */ - static Configuration closeJobClient(JobClient jobClient, File savepointDir) { - Configuration conf = new Configuration(); - if (jobClient != null) { - if (savepointDir != null) { - // Stop with savepoint - jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); - // Wait until the savepoint is created and the job has been stopped - Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); - conf.set( - SavepointConfigOptions.SAVEPOINT_PATH, - savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath()); - } else { - jobClient.cancel(); - } - - // Wait until the job has been stopped - Awaitility.await().until(() -> jobClient.getJobStatus().get().isTerminalState()); - return conf; - } - - return null; - } - - /** - * Close the {@link JobClient} and wait for the job closure. - * - * @param jobClient the job to close - */ - static void closeJobClient(JobClient jobClient) { - closeJobClient(jobClient, null); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java index 679b3ec508a2..eff32fcfa118 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java @@ -44,7 +44,7 @@ import org.jetbrains.annotations.Nullable; /** Testing source implementation for Flink sources which can be triggered manually. */ -class ManualSource +public class ManualSource implements Source, ResultTypeQueryable { @@ -65,7 +65,7 @@ class ManualSource * @param env to register the source * @param type of the events returned by the source */ - ManualSource(StreamExecutionEnvironment env, TypeInformation type) { + public ManualSource(StreamExecutionEnvironment env, TypeInformation type) { this.type = type; this.env = env; this.index = numSources++; @@ -78,7 +78,7 @@ class ManualSource * * @param event to emit */ - void sendRecord(T event) { + public void sendRecord(T event) { this.sendInternal(Tuple2.of(event, null)); } @@ -88,7 +88,7 @@ void sendRecord(T event) { * @param event to emit * @param eventTime of the event */ - void sendRecord(T event, long eventTime) { + public void sendRecord(T event, long eventTime) { this.sendInternal(Tuple2.of(event, eventTime)); } @@ -97,7 +97,7 @@ void sendRecord(T event, long eventTime) { * * @param timeStamp of the watermark */ - void sendWatermark(long timeStamp) { + public void sendWatermark(long timeStamp) { this.sendInternal(Tuple2.of(null, timeStamp)); } @@ -112,7 +112,7 @@ void markFinished() { * * @return the stream emitted by this source */ - DataStream dataStream() { + public DataStream dataStream() { if (this.stream == null) { this.stream = this.env diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java index 7a523035b7fb..ed66ff3df076 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; @@ -38,10 +39,24 @@ public class MetricsReporterFactoryForTests implements MetricReporterFactory { private static final TestMetricsReporter INSTANCE = new TestMetricsReporter(); - private static final Pattern FULL_METRIC_NAME = + private static final Pattern TASK_METRIC_NAME = Pattern.compile( "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + TableMaintenanceMetrics.GROUP_KEY + + "\\." + + TableMaintenanceMetrics.TABLE_NAME_KEY + + "\\.([^.]+)\\." + + TableMaintenanceMetrics.TASK_NAME_KEY + + "\\.([^.]+)\\." + + TableMaintenanceMetrics.TASK_INDEX_KEY + + "\\.([^.]+)\\.([^.]+)"); + + private static final Pattern MAIN_METRIC_NAME = + Pattern.compile( + "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + + TableMaintenanceMetrics.GROUP_KEY + + "\\." + + TableMaintenanceMetrics.TABLE_NAME_KEY + "\\.([^.]+)\\.([^.]+)"); private static Map counters = Maps.newConcurrentMap(); @@ -72,20 +87,26 @@ public static void reset() { gauges = Maps.newConcurrentMap(); } - public static Long counter(String name) { - return counterValues().get(name); + public static Long counter(List parts) { + return counterValues().get(longName(parts)); } - public static Long gauge(String name) { - return gaugeValues().get(name); + public static Long gauge(List parts) { + return gaugeValues().get(longName(parts)); } - public static void assertGauges(Map expected) { - assertThat(filter(gaugeValues(), expected)).isEqualTo(filter(expected, expected)); + public static void assertGauges(Map, Long> expected) { + Map transformed = + expected.entrySet().stream() + .collect(Collectors.toMap(k -> longName(k.getKey()), Map.Entry::getValue)); + assertThat(filter(gaugeValues(), transformed)).isEqualTo(filter(transformed, transformed)); } - public static void assertCounters(Map expected) { - assertThat(filter(counterValues(), expected)).isEqualTo(filter(expected, expected)); + public static void assertCounters(Map, Long> expected) { + Map transformed = + expected.entrySet().stream() + .collect(Collectors.toMap(k -> longName(k.getKey()), Map.Entry::getValue)); + assertThat(filter(counterValues(), transformed)).isEqualTo(filter(transformed, transformed)); } private static Map gaugeValues() { @@ -113,12 +134,30 @@ private static Map filter(Map original, Map parts) { + return parts.stream().map(s -> s.replaceAll("\\.", "_")).collect(Collectors.joining(".")); } private static class TestMetricsReporter implements MetricReporter { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 225853086545..5e05f40e53cf 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -19,27 +19,62 @@ package org.apache.iceberg.flink.maintenance.operator; import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; +import java.io.IOException; +import java.nio.file.Path; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.flink.streaming.api.transformations.SinkTransformation; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.iceberg.flink.FlinkCatalogFactory; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; -class OperatorTestBase { +public class OperatorTestBase { private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; - private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); - private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); + private static final Schema SCHEMA_WITH_PRIMARY_KEY = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())), + ImmutableMap.of(), + ImmutableSet.of(SimpleDataUtil.SCHEMA.columns().get(0).fieldId())); - static final String TABLE_NAME = "test_table"; + protected static final String UID_SUFFIX = "UID-Dummy"; + protected static final String SLOT_SHARING_GROUP = "SlotSharingGroup"; + protected static final TriggerLockFactory LOCK_FACTORY = new MemoryLockFactory(); + + public static final String IGNORED_OPERATOR_NAME = "Ignore"; + + static final long EVENT_TIME = 10L; + static final long EVENT_TIME_2 = 11L; + protected static final String DUMMY_TASK_NAME = "dummyTask"; + protected static final String DUMMY_TABLE_NAME = "dummyTable"; @RegisterExtension protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = @@ -50,43 +85,57 @@ class OperatorTestBase { .setConfiguration(config()) .build()); + @TempDir private Path warehouseDir; + @RegisterExtension - final FlinkSqlExtension sql = - new FlinkSqlExtension( - "catalog", - ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), - "db"); + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - private static Configuration config() { - Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); - MetricOptions.forReporter(config, "test_reporter") - .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); - return config; + @BeforeEach + void before() { + LOCK_FACTORY.open(); + MetricsReporterFactoryForTests.reset(); } - protected static TriggerLockFactory lockFactory() { - return new TriggerLockFactory() { - @Override - public void open() { - MAINTENANCE_LOCK.unlock(); - RECOVERY_LOCK.unlock(); - } + @AfterEach + void after() throws IOException { + LOCK_FACTORY.close(); + } - @Override - public Lock createLock() { - return MAINTENANCE_LOCK; - } + protected static Table createTable() { + return CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("flink.max-continuous-empty-commits", "100000")); + } - @Override - public Lock createRecoveryLock() { - return RECOVERY_LOCK; - } + protected static Table createTableWithDelete() { + return CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA_WITH_PRIMARY_KEY, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("format-version", "2", "write.upsert.enabled", "true")); + } - @Override - public void close() { - // do nothing - } - }; + protected void insert(Table table, Integer id, String data) throws IOException { + new GenericAppenderHelper(table, FileFormat.PARQUET, warehouseDir) + .appendToTable(Lists.newArrayList(SimpleDataUtil.createRecord(id, data))); + table.refresh(); + } + + protected void dropTable() { + CATALOG_EXTENSION.catalogLoader().loadCatalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + protected TableLoader tableLoader() { + return CATALOG_EXTENSION.tableLoader(); } /** @@ -98,7 +147,7 @@ public void close() { * stop without a savepoint. * @return configuration for restarting the job from the savepoint */ - public static Configuration closeJobClient(JobClient jobClient, File savepointDir) { + protected static Configuration closeJobClient(JobClient jobClient, File savepointDir) { Configuration conf = new Configuration(); if (jobClient != null) { if (savepointDir != null) { @@ -126,12 +175,45 @@ public static Configuration closeJobClient(JobClient jobClient, File savepointDi * * @param jobClient the job to close */ - public static void closeJobClient(JobClient jobClient) { + protected static void closeJobClient(JobClient jobClient) { closeJobClient(jobClient, null); } + protected static void checkUidsAreSet(StreamExecutionEnvironment env, String uidSuffix) { + env.getTransformations().stream() + .filter( + t -> !(t instanceof SinkTransformation) && !(t.getName().equals(IGNORED_OPERATOR_NAME))) + .forEach( + transformation -> { + assertThat(transformation.getUid()).isNotNull(); + if (uidSuffix != null) { + assertThat(transformation.getUid()).contains(UID_SUFFIX); + } + }); + } + + protected static void checkSlotSharingGroupsAreSet(StreamExecutionEnvironment env, String name) { + String nameToCheck = name != null ? name : StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP; + + env.getTransformations().stream() + .filter( + t -> !(t instanceof SinkTransformation) && !(t.getName().equals(IGNORED_OPERATOR_NAME))) + .forEach( + t -> { + assertThat(t.getSlotSharingGroup()).isPresent(); + assertThat(t.getSlotSharingGroup().get().getName()).isEqualTo(nameToCheck); + }); + } + + private static Configuration config() { + Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + MetricOptions.forReporter(config, "test_reporter") + .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); + return config; + } + private static class MemoryLock implements TriggerLockFactory.Lock { - boolean locked = false; + volatile boolean locked = false; @Override public boolean tryLock() { @@ -153,4 +235,30 @@ public void unlock() { locked = false; } } + + private static class MemoryLockFactory implements TriggerLockFactory { + private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); + + @Override + public void open() { + MAINTENANCE_LOCK.unlock(); + RECOVERY_LOCK.unlock(); + } + + @Override + public Lock createLock() { + return MAINTENANCE_LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // do nothing + } + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java new file mode 100644 index 000000000000..d70c4aafd59a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDeleteFilesProcessor.java @@ -0,0 +1,116 @@ +/* + * 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.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestDeleteFilesProcessor extends OperatorTestBase { + private static final String DUMMY_FILE_NAME = "dummy"; + private static final Set TABLE_FILES = + ImmutableSet.of( + "metadata/v1.metadata.json", + "metadata/version-hint.text", + "metadata/.version-hint.text.crc", + "metadata/.v1.metadata.json.crc"); + + private Table table; + + @BeforeEach + void before() { + this.table = createTable(); + } + + @Test + void testDelete() throws Exception { + // Write an extra file + Path dummyFile = Path.of(tablePath(table).toString(), DUMMY_FILE_NAME); + Files.write(dummyFile, "DUMMY".getBytes(StandardCharsets.UTF_8)); + + Set files = listFiles(table); + assertThat(files) + .containsAll(TABLE_FILES) + .contains(DUMMY_FILE_NAME) + .hasSize(TABLE_FILES.size() + 1); + + deleteFile(tableLoader(), dummyFile.toString()); + + assertThat(listFiles(table)).isEqualTo(TABLE_FILES); + } + + @Test + void testDeleteMissingFile() throws Exception { + Path dummyFile = + FileSystems.getDefault().getPath(table.location().substring(5), DUMMY_FILE_NAME); + + deleteFile(tableLoader(), dummyFile.toString()); + + assertThat(listFiles(table)).isEqualTo(TABLE_FILES); + } + + @Test + void testInvalidURIScheme() throws Exception { + deleteFile(tableLoader(), "wrong://"); + + assertThat(listFiles(table)).isEqualTo(TABLE_FILES); + } + + private void deleteFile(TableLoader tableLoader, String fileName) throws Exception { + tableLoader().open(); + try (OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>( + new DeleteFilesProcessor(0, DUMMY_TASK_NAME, tableLoader.loadTable(), 10), + StringSerializer.INSTANCE)) { + testHarness.open(); + testHarness.processElement(fileName, System.currentTimeMillis()); + testHarness.processWatermark(EVENT_TIME); + testHarness.endInput(); + } + } + + private static Path tablePath(Table table) { + return FileSystems.getDefault().getPath(table.location().substring(5)); + } + + private static Set listFiles(Table table) throws IOException { + String tableRootPath = TestFixtures.TABLE_IDENTIFIER.toString().replace(".", "/"); + return Files.find( + tablePath(table), Integer.MAX_VALUE, (filePath, fileAttr) -> fileAttr.isRegularFile()) + .map( + p -> + p.toString() + .substring(p.toString().indexOf(tableRootPath) + tableRootPath.length() + 1)) + .collect(Collectors.toSet()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java new file mode 100644 index 000000000000..d312fc312c99 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestExpireSnapshotsProcessor.java @@ -0,0 +1,80 @@ +/* + * 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.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Queue; +import java.util.Set; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +class TestExpireSnapshotsProcessor extends OperatorTestBase { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testExpire(boolean success) throws Exception { + Table table = createTable(); + insert(table, 1, "a"); + insert(table, 2, "b"); + + List actual; + Queue> deletes; + try (OneInputStreamOperatorTestHarness testHarness = + ProcessFunctionTestHarnesses.forProcessFunction( + new ExpireSnapshotsProcessor(tableLoader(), 0L, 1, 10))) { + testHarness.open(); + + if (!success) { + // Cause an exception + dropTable(); + } + + testHarness.processElement(Trigger.create(10, 11), System.currentTimeMillis()); + deletes = testHarness.getSideOutput(ExpireSnapshotsProcessor.DELETE_STREAM); + actual = testHarness.extractOutputValues(); + } + + assertThat(actual).hasSize(1); + TaskResult result = actual.get(0); + assertThat(result.startEpoch()).isEqualTo(10); + assertThat(result.taskIndex()).isEqualTo(11); + assertThat(result.success()).isEqualTo(success); + + if (success) { + assertThat(result.exceptions()).isNotNull().isEmpty(); + + table.refresh(); + Set snapshots = Sets.newHashSet(table.snapshots()); + assertThat(snapshots).hasSize(1); + assertThat(deletes).hasSize(1); + } else { + assertThat(result.exceptions()).isNotNull().hasSize(1); + assertThat(deletes).isNull(); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java index cffcc4eb0471..4e4de9e78e9e 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.FAILED_TASK_COUNTER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.LAST_RUN_DURATION_MS; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.util.Collection; +import java.util.List; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.sink2.Committer; @@ -43,6 +44,9 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; +import org.apache.iceberg.flink.maintenance.api.TaskResult; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; @@ -53,7 +57,7 @@ @Timeout(value = 10) class TestLockRemover extends OperatorTestBase { - private static final String[] TASKS = new String[] {"task0", "task1"}; + private static final String[] TASKS = new String[] {"task0", "task1", "task2"}; private static final TriggerLockFactory.Lock LOCK = new TestingLock(); private static final TriggerLockFactory.Lock RECOVERY_LOCK = new TestingLock(); @@ -71,9 +75,9 @@ void testProcess() throws Exception { source .dataStream() .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + new LockRemover(DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS))) .setParallelism(1); JobClient jobClient = null; @@ -128,37 +132,79 @@ void testMetrics() throws Exception { source .dataStream() .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + new LockRemover(DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS))) .setParallelism(1); JobClient jobClient = null; + long time = System.currentTimeMillis(); try { jobClient = env.executeAsync(); // Start the 2 successful and one failed result trigger for task1, and 3 successful for task2 - processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(1, 1L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(1, 2L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(0, 3L, false, Lists.newArrayList())); - processAndCheck(source, new TaskResult(0, 4L, true, Lists.newArrayList())); - processAndCheck(source, new TaskResult(1, 5L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, time, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 0L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 0L, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, time, false, Lists.newArrayList())); + processAndCheck(source, new TaskResult(0, time, true, Lists.newArrayList())); + processAndCheck(source, new TaskResult(1, 0L, true, Lists.newArrayList())); Awaitility.await() .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER) + ImmutableList.of( + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + TASKS[1], + "1", + SUCCEEDED_TASK_COUNTER)) .equals(3L)); // Final check all the counters MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() - .put(DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER, 2L) - .put(DUMMY_NAME + "." + TASKS[0] + "." + FAILED_TASK_COUNTER, 1L) - .put(DUMMY_NAME + "." + TASKS[1] + "." + SUCCEEDED_TASK_COUNTER, 3L) - .put(DUMMY_NAME + "." + TASKS[1] + "." + FAILED_TASK_COUNTER, 0L) + new ImmutableMap.Builder, Long>() + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[0], "0", SUCCEEDED_TASK_COUNTER), + 2L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[0], "0", FAILED_TASK_COUNTER), + 1L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[1], "1", SUCCEEDED_TASK_COUNTER), + 3L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[1], "1", FAILED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[2], "2", SUCCEEDED_TASK_COUNTER), + 0L) + .put( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[2], "2", FAILED_TASK_COUNTER), + 0L) .build()); + + assertThat( + MetricsReporterFactoryForTests.gauge( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[0], "0", LAST_RUN_DURATION_MS))) + .isPositive(); + assertThat( + MetricsReporterFactoryForTests.gauge( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[1], "1", LAST_RUN_DURATION_MS))) + .isGreaterThan(time); + assertThat( + MetricsReporterFactoryForTests.gauge( + ImmutableList.of( + DUMMY_TASK_NAME, DUMMY_TABLE_NAME, TASKS[2], "2", LAST_RUN_DURATION_MS))) + .isZero(); } finally { closeJobClient(jobClient); } @@ -181,9 +227,10 @@ void testRecovery() throws Exception { .dataStream() .union(source2.dataStream()) .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) + new LockRemover( + DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) .setParallelism(1); JobClient jobClient = null; @@ -201,7 +248,12 @@ void testRecovery() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + ImmutableList.of( + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + TASKS[0], + "0", + SUCCEEDED_TASK_COUNTER)) .equals(2L)); // We did not remove the recovery lock, as no watermark received from the other source @@ -223,20 +275,21 @@ private void processAndCheck(ManualSource source, TaskResult input) private void processAndCheck( ManualSource source, TaskResult input, String counterPrefix) { + List counterKey = + ImmutableList.of( + (counterPrefix != null ? counterPrefix : "") + DUMMY_TASK_NAME, + DUMMY_TABLE_NAME, + TASKS[input.taskIndex()], + String.valueOf(input.taskIndex()), + input.success() ? SUCCEEDED_TASK_COUNTER : FAILED_TASK_COUNTER); + Long counterValue = MetricsReporterFactoryForTests.counter(counterKey); + Long expected = counterValue != null ? counterValue + 1 : 1L; + source.sendRecord(input); source.sendWatermark(input.startEpoch()); - String counterName = - (counterPrefix != null ? counterPrefix : "") - .concat( - input.success() - ? DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + SUCCEEDED_TASK_COUNTER - : DUMMY_NAME + "." + TASKS[input.taskIndex()] + "." + FAILED_TASK_COUNTER); - Long counterValue = MetricsReporterFactoryForTests.counter(counterName); - Long expected = counterValue != null ? counterValue + 1 : 1L; - Awaitility.await() - .until(() -> expected.equals(MetricsReporterFactoryForTests.counter(counterName))); + .until(() -> expected.equals(MetricsReporterFactoryForTests.counter(counterKey))); } private static class TestingLockFactory implements TriggerLockFactory { @@ -370,9 +423,10 @@ public void flatMap( } }) .transform( - DUMMY_NAME, + DUMMY_TASK_NAME, TypeInformation.of(Void.class), - new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); + new LockRemover( + DUMMY_TABLE_NAME, new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); } } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index 3aee05322561..c561c7054eae 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; +import java.io.IOException; import java.time.Duration; import java.util.List; import java.util.concurrent.atomic.AtomicReference; @@ -60,40 +61,27 @@ class TestMonitorSource extends OperatorTestBase { @ParameterizedTest @ValueSource(booleans = {true, false}) - void testChangeReaderIterator(boolean withDelete) { - if (withDelete) { - sql.exec( - "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) WITH ('format-version'='2', 'write.upsert.enabled'='true')", - TABLE_NAME); - } else { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - } - - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); - Table table = tableLoader.loadTable(); + void testChangeReaderIterator(boolean withDelete) throws IOException { + Table table = withDelete ? createTableWithDelete() : createTable(); MonitorSource.TableChangeIterator iterator = - new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); + new MonitorSource.TableChangeIterator(tableLoader(), null, Long.MAX_VALUE); // For an empty table we get an empty result assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); // Add a single commit and get back the commit data in the event - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); - table.refresh(); + insert(table, 1, "a"); TableChange expected = tableChangeWithLastSnapshot(table, TableChange.empty()); assertThat(iterator.next()).isEqualTo(expected); // Make sure that consecutive calls do not return the data again assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); // Add two more commits, but fetch the data in one loop - sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); - table.refresh(); + insert(table, 2, "b"); expected = tableChangeWithLastSnapshot(table, TableChange.empty()); - sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); - table.refresh(); + insert(table, 3, "c"); expected = tableChangeWithLastSnapshot(table, expected); assertThat(iterator.next()).isEqualTo(expected); @@ -106,17 +94,11 @@ void testChangeReaderIterator(boolean withDelete) { */ @Test void testSource() throws Exception { - sql.exec( - "CREATE TABLE %s (id int, data varchar) " - + "WITH ('flink.max-continuous-empty-commits'='100000')", - TABLE_NAME); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); - Table table = tableLoader.loadTable(); + Table table = createTable(); DataStream events = env.fromSource( - new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + new MonitorSource(tableLoader(), HIGH_RATE, Long.MAX_VALUE), WatermarkStrategy.noWatermarks(), "TableChangeSource") .forceNonParallel(); @@ -176,8 +158,9 @@ void testSource() throws Exception { /** Check that the {@link MonitorSource} operator state is restored correctly. */ @Test void testStateRestore(@TempDir File savepointDir) throws Exception { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + Table table = createTable(); + insert(table, 1, "a"); + TableLoader tableLoader = tableLoader(); Configuration config = new Configuration(); config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); @@ -185,8 +168,6 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); env.enableCheckpointing(1000); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); DataStream events = env.fromSource( new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), @@ -268,14 +249,12 @@ void testStateRestore(@TempDir File savepointDir) throws Exception { @Test void testNotOneParallelismThrows() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + createTable(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); env.fromSource( - new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + new MonitorSource(tableLoader(), HIGH_RATE, Long.MAX_VALUE), WatermarkStrategy.noWatermarks(), "TableChangeSource") .setParallelism(2) @@ -289,14 +268,13 @@ void testNotOneParallelismThrows() { } @Test - void testMaxReadBack() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + void testMaxReadBack() throws IOException { + Table table = createTable(); + insert(table, 1, "a"); + insert(table, 2, "b"); + insert(table, 3, "c"); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); + TableLoader tableLoader = tableLoader(); MonitorSource.TableChangeIterator iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 1); @@ -316,12 +294,11 @@ void testMaxReadBack() { } @Test - void testSkipReplace() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + void testSkipReplace() throws IOException { + Table table = createTable(); + insert(table, 1, "a"); - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); - tableLoader.open(); + TableLoader tableLoader = tableLoader(); MonitorSource.TableChangeIterator iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); @@ -330,7 +307,6 @@ void testSkipReplace() { assertThat(iterator.next().commitCount()).isEqualTo(1); // Create a DataOperations.REPLACE snapshot - Table table = tableLoader.loadTable(); DataFile dataFile = table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java index fba4a12d9c6b..a70d27279460 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -18,19 +18,13 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME; -import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME_2; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; -import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.GROUP_VALUE_DEFAULT; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED; import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; import static org.assertj.core.api.Assertions.assertThat; -import java.io.IOException; import java.time.Duration; -import java.util.Iterator; import java.util.List; import java.util.stream.Stream; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -41,13 +35,14 @@ import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.maintenance.api.Trigger; +import org.apache.iceberg.flink.maintenance.api.TriggerLockFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -56,34 +51,24 @@ class TestTriggerManager extends OperatorTestBase { private static final long DELAY = 10L; - private static final String NAME_1 = "name1"; - private static final String NAME_2 = "name2"; + private static final String[] TASKS = new String[] {"task0", "task1"}; private long processingTime = 0L; - private TriggerLockFactory lockFactory; private TriggerLockFactory.Lock lock; private TriggerLockFactory.Lock recoveringLock; + private String tableName; @BeforeEach void before() { - sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); - this.lockFactory = lockFactory(); - lockFactory.open(); - this.lock = lockFactory.createLock(); - this.recoveringLock = lockFactory.createRecoveryLock(); - lock.unlock(); - recoveringLock.unlock(); - MetricsReporterFactoryForTests.reset(); - } - - @AfterEach - void after() throws IOException { - lockFactory.close(); + Table table = createTable(); + this.lock = LOCK_FACTORY.createLock(); + this.recoveringLock = LOCK_FACTORY.createRecoveryLock(); + this.tableName = table.name(); } @Test void testCommitCount() throws Exception { TriggerManager manager = - manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().commitCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -104,8 +89,7 @@ void testCommitCount() throws Exception { @Test void testDataFileCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().dataFileCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().dataFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -126,9 +110,7 @@ void testDataFileCount() throws Exception { @Test void testDataFileSizeInBytes() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -147,9 +129,7 @@ void testDataFileSizeInBytes() throws Exception { @Test void testPosDeleteFileCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -170,9 +150,7 @@ void testPosDeleteFileCount() throws Exception { @Test void testPosDeleteRecordCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -196,9 +174,7 @@ void testPosDeleteRecordCount() throws Exception { @Test void testEqDeleteFileCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -219,9 +195,7 @@ void testEqDeleteFileCount() throws Exception { @Test void testEqDeleteRecordCount() throws Exception { TriggerManager manager = - manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); + manager(tableLoader(), new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -241,8 +215,7 @@ void testEqDeleteRecordCount() throws Exception { void testTimeout() throws Exception { TriggerManager manager = manager( - sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); + tableLoader(), new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); @@ -281,7 +254,7 @@ void testTimeout() throws Exception { @Test void testStateRestore() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader); OperatorSubtaskState state; try (KeyedOneInputStreamOperatorTestHarness testHarness = @@ -319,7 +292,7 @@ void testStateRestore() throws Exception { @Test void testMinFireDelay() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader, DELAY, 1); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { @@ -339,7 +312,7 @@ void testMinFireDelay() throws Exception { @Test void testLockCheckDelay() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader, 1, DELAY); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { @@ -372,7 +345,7 @@ void testLockCheckDelay() throws Exception { @ParameterizedTest @MethodSource("parametersForTestRecovery") void testRecovery(boolean locked, boolean runningTask) throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); TriggerManager manager = manager(tableLoader); OperatorSubtaskState state; try (KeyedOneInputStreamOperatorTestHarness testHarness = @@ -423,18 +396,14 @@ void testRecovery(boolean locked, boolean runningTask) throws Exception { ++processingTime; testHarness.setProcessingTime(processingTime); // Releasing lock will create a new snapshot, and we receive this in the trigger - expected.add( - Trigger.create( - processingTime, - (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()), - 0)); + expected.add(Trigger.create(processingTime, 0)); assertTriggers(testHarness.extractOutputValues(), expected); } } @Test void testTriggerMetrics() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); ManualSource source = @@ -444,8 +413,8 @@ void testTriggerMetrics() throws Exception { TriggerManager manager = new TriggerManager( tableLoader, - lockFactory, - Lists.newArrayList(NAME_1, NAME_2), + LOCK_FACTORY, + Lists.newArrayList(TASKS), Lists.newArrayList( new TriggerEvaluator.Builder().commitCount(2).build(), new TriggerEvaluator.Builder().commitCount(4).build()), @@ -455,7 +424,7 @@ void testTriggerMetrics() throws Exception { .dataStream() .keyBy(unused -> true) .process(manager) - .name(DUMMY_NAME) + .name(DUMMY_TASK_NAME) .forceNonParallel() .sinkTo(sink); @@ -471,7 +440,7 @@ void testTriggerMetrics() throws Exception { () -> { Long notingCounter = MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER); + ImmutableList.of(DUMMY_TASK_NAME, tableName, NOTHING_TO_TRIGGER)); return notingCounter != null && notingCounter.equals(1L); }); @@ -480,7 +449,8 @@ void testTriggerMetrics() throws Exception { // Wait until we receive the trigger assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter( + ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED))) .isEqualTo(1L); lock.unlock(); @@ -492,20 +462,22 @@ void testTriggerMetrics() throws Exception { assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); lock.unlock(); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter( + ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED))) .isEqualTo(2L); assertThat( - MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED)) + MetricsReporterFactoryForTests.counter( + ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[1], "1", TRIGGERED))) .isEqualTo(1L); // Final check all the counters MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, -1L) - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, -1L) - .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 2L) - .put(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED, 1L) - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 1L) + new ImmutableMap.Builder, Long>() + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, RATE_LIMITER_TRIGGERED), -1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, CONCURRENT_RUN_THROTTLED), -1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED), 2L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[1], "1", TRIGGERED), 1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, NOTHING_TO_TRIGGER), 1L) .build()); } finally { closeJobClient(jobClient); @@ -514,7 +486,7 @@ void testTriggerMetrics() throws Exception { @Test void testRateLimiterMetrics() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); ManualSource source = @@ -527,7 +499,7 @@ void testRateLimiterMetrics() throws Exception { .dataStream() .keyBy(unused -> true) .process(manager) - .name(DUMMY_NAME) + .name(DUMMY_TASK_NAME) .forceNonParallel() .sinkTo(sink); @@ -548,7 +520,7 @@ void testRateLimiterMetrics() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED) + ImmutableList.of(DUMMY_TASK_NAME, tableName, RATE_LIMITER_TRIGGERED)) .equals(1L)); // Final check all the counters @@ -560,7 +532,7 @@ void testRateLimiterMetrics() throws Exception { @Test void testConcurrentRunMetrics() throws Exception { - TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TableLoader tableLoader = tableLoader(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); ManualSource source = @@ -573,7 +545,7 @@ void testConcurrentRunMetrics() throws Exception { .dataStream() .keyBy(unused -> true) .process(manager) - .name(DUMMY_NAME) + .name(DUMMY_TASK_NAME) .forceNonParallel() .sinkTo(sink); @@ -591,7 +563,7 @@ void testConcurrentRunMetrics() throws Exception { .until( () -> MetricsReporterFactoryForTests.counter( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED) + ImmutableList.of(DUMMY_TASK_NAME, tableName, CONCURRENT_RUN_THROTTLED)) .equals(1L)); // Final check all the counters @@ -611,15 +583,15 @@ private static Stream parametersForTestRecovery() { private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) { MetricsReporterFactoryForTests.assertCounters( - new ImmutableMap.Builder() + new ImmutableMap.Builder, Long>() .put( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, + ImmutableList.of(DUMMY_TASK_NAME, tableName, RATE_LIMITER_TRIGGERED), rateLimiterTrigger) .put( - DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, + ImmutableList.of(DUMMY_TASK_NAME, tableName, CONCURRENT_RUN_THROTTLED), concurrentRunTrigger) - .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 1L) - .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, TASKS[0], "0", TRIGGERED), 1L) + .put(ImmutableList.of(DUMMY_TASK_NAME, tableName, NOTHING_TO_TRIGGER), 0L) .build()); } @@ -644,15 +616,20 @@ private void addEventAndCheckResult( private TriggerManager manager(TableLoader tableLoader, TriggerEvaluator evaluator) { return new TriggerManager( - tableLoader, lockFactory, Lists.newArrayList(NAME_1), Lists.newArrayList(evaluator), 1, 1); + tableLoader, + LOCK_FACTORY, + Lists.newArrayList(TASKS[0]), + Lists.newArrayList(evaluator), + 1, + 1); } private TriggerManager manager( TableLoader tableLoader, long minFireDelayMs, long lockCheckDelayMs) { return new TriggerManager( tableLoader, - lockFactory, - Lists.newArrayList(NAME_1), + LOCK_FACTORY, + Lists.newArrayList(TASKS[0]), Lists.newArrayList(new TriggerEvaluator.Builder().commitCount(2).build()), minFireDelayMs, lockCheckDelayMs); @@ -670,17 +647,6 @@ private static void assertTriggers(List expected, List actual) assertThat(actualTrigger.timestamp()).isEqualTo(expectedTrigger.timestamp()); assertThat(actualTrigger.taskId()).isEqualTo(expectedTrigger.taskId()); assertThat(actualTrigger.isRecovery()).isEqualTo(expectedTrigger.isRecovery()); - if (expectedTrigger.table() == null) { - assertThat(actualTrigger.table()).isNull(); - } else { - Iterator expectedSnapshots = expectedTrigger.table().snapshots().iterator(); - Iterator actualSnapshots = actualTrigger.table().snapshots().iterator(); - while (expectedSnapshots.hasNext()) { - assertThat(actualSnapshots.hasNext()).isTrue(); - assertThat(expectedSnapshots.next().snapshotId()) - .isEqualTo(actualSnapshots.next().snapshotId()); - } - } } } } From fff9ec3bbc322080da6363b657415b039c0e92a0 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 8 Nov 2024 14:36:27 +0800 Subject: [PATCH 114/313] Docs: Fix format of verifying release candidate with Flink (#11487) --- site/docs/how-to-release.md | 1 + 1 file changed, 1 insertion(+) diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index 85ecd64e51f7..be17d9495d5b 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -441,6 +441,7 @@ sql-client.sh embedded \ -j iceberg-flink-runtime-1.20-{{ icebergVersion }}.jar \ -j flink-connector-hive_2.12-1.20.jar \ shell +``` ## Voting From 166edc7298825321f677e1e70cf88d7249e8035c Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 8 Nov 2024 18:09:49 +0100 Subject: [PATCH 115/313] Core: Support DVs in DeleteLoader (#11481) --- .../apache/iceberg/util/ContentFileUtil.java | 5 + .../apache/iceberg/data/BaseDeleteLoader.java | 82 +++++++ .../org/apache/iceberg/data/DeleteLoader.java | 5 +- .../org/apache/iceberg/io/TestDVWriters.java | 221 +++++++++++++++++- 4 files changed, 310 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java index 9e4a65be02ae..beffd3a955c9 100644 --- a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java @@ -26,6 +26,7 @@ import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; @@ -94,6 +95,10 @@ public static boolean isDV(DeleteFile deleteFile) { return deleteFile.format() == FileFormat.PUFFIN; } + public static boolean containsSingleDV(Iterable deleteFiles) { + return Iterables.size(deleteFiles) == 1 && Iterables.all(deleteFiles, ContentFileUtil::isDV); + } + public static String dvDesc(DeleteFile deleteFile) { return String.format( "DV{location=%s, offset=%s, length=%s, referencedDataFile=%s}", diff --git a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java index 8a1ebf95abeb..796f4f92be33 100644 --- a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java +++ b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java @@ -42,15 +42,20 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DeleteSchemaUtil; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.RangeReadable; +import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.orc.OrcRowReader; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.parquet.ParquetValueReader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.apache.iceberg.relocated.com.google.common.math.LongMath; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.CharSequenceMap; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; @@ -143,9 +148,48 @@ private Iterable materialize(CloseableIterable iterable) { } } + /** + * Loads the content of a deletion vector or position delete files for a given data file path into + * a position index. + * + *

    The deletion vector is currently loaded without caching as the existing Puffin reader + * requires at least 3 requests to fetch the entire file. Caching a single deletion vector may + * only be useful when multiple data file splits are processed on the same node, which is unlikely + * as task locality is not guaranteed. + * + *

    For position delete files, however, there is no efficient way to read deletes for a + * particular data file. Therefore, caching may be more effective as such delete files potentially + * apply to many data files, especially in unpartitioned tables and tables with deep partitions. + * If a position delete file qualifies for caching, this method will attempt to cache a position + * index for each referenced data file. + * + * @param deleteFiles a deletion vector or position delete files + * @param filePath the data file path for which to load deletes + * @return a position delete index for the provided data file path + */ @Override public PositionDeleteIndex loadPositionDeletes( Iterable deleteFiles, CharSequence filePath) { + if (ContentFileUtil.containsSingleDV(deleteFiles)) { + DeleteFile dv = Iterables.getOnlyElement(deleteFiles); + validateDV(dv, filePath); + return readDV(dv); + } else { + return getOrReadPosDeletes(deleteFiles, filePath); + } + } + + private PositionDeleteIndex readDV(DeleteFile dv) { + LOG.trace("Opening DV file {}", dv.location()); + InputFile inputFile = loadInputFile.apply(dv); + long offset = dv.contentOffset(); + int length = dv.contentSizeInBytes().intValue(); + byte[] bytes = readBytes(inputFile, offset, length); + return PositionDeleteIndex.deserialize(bytes, dv); + } + + private PositionDeleteIndex getOrReadPosDeletes( + Iterable deleteFiles, CharSequence filePath) { Iterable deletes = execute(deleteFiles, deleteFile -> getOrReadPosDeletes(deleteFile, filePath)); return PositionDeleteIndexUtil.merge(deletes); @@ -259,4 +303,42 @@ private long estimateEqDeletesSize(DeleteFile deleteFile, Schema projection) { private int estimateRecordSize(Schema schema) { return schema.columns().stream().mapToInt(TypeUtil::estimateSize).sum(); } + + private void validateDV(DeleteFile dv, CharSequence filePath) { + Preconditions.checkArgument( + dv.contentOffset() != null, + "Invalid DV, offset cannot be null: %s", + ContentFileUtil.dvDesc(dv)); + Preconditions.checkArgument( + dv.contentSizeInBytes() != null, + "Invalid DV, length is null: %s", + ContentFileUtil.dvDesc(dv)); + Preconditions.checkArgument( + dv.contentSizeInBytes() <= Integer.MAX_VALUE, + "Can't read DV larger than 2GB: %s", + dv.contentSizeInBytes()); + Preconditions.checkArgument( + filePath.toString().equals(dv.referencedDataFile()), + "DV is expected to reference %s, not %s", + filePath, + dv.referencedDataFile()); + } + + private static byte[] readBytes(InputFile inputFile, long offset, int length) { + try (SeekableInputStream stream = inputFile.newStream()) { + byte[] bytes = new byte[length]; + + if (stream instanceof RangeReadable) { + RangeReadable rangeReadable = (RangeReadable) stream; + rangeReadable.readFully(offset, bytes); + } else { + stream.seek(offset); + ByteStreams.readFully(stream, bytes); + } + + return bytes; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } } diff --git a/data/src/main/java/org/apache/iceberg/data/DeleteLoader.java b/data/src/main/java/org/apache/iceberg/data/DeleteLoader.java index 07bdce6d836f..0fc0b93f7350 100644 --- a/data/src/main/java/org/apache/iceberg/data/DeleteLoader.java +++ b/data/src/main/java/org/apache/iceberg/data/DeleteLoader.java @@ -35,9 +35,10 @@ public interface DeleteLoader { StructLikeSet loadEqualityDeletes(Iterable deleteFiles, Schema projection); /** - * Loads the content of position delete files for a given data file path into a position index. + * Loads the content of a deletion vector or position delete files for a given data file path into + * a position index. * - * @param deleteFiles position delete files + * @param deleteFiles a deletion vector or position delete files * @param filePath the data file path for which to load deletes * @return a position delete index for the provided data file path */ diff --git a/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java b/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java index ce742b1c4685..23e0090ca49f 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java @@ -19,6 +19,7 @@ package org.apache.iceberg.io; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; import java.util.Arrays; @@ -28,17 +29,25 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.BaseDeleteLoader; import org.apache.iceberg.data.DeleteLoader; import org.apache.iceberg.deletes.BaseDVFileWriter; import org.apache.iceberg.deletes.DVFileWriter; +import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeSet; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -49,10 +58,11 @@ public abstract class TestDVWriters extends WriterTestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(new Object[] {3}); + return Arrays.asList(new Object[] {2, 3}); } private OutputFileFactory fileFactory = null; + private OutputFileFactory parquetFileFactory = null; protected abstract StructLikeSet toSet(Iterable records); @@ -65,10 +75,14 @@ protected FileFormat dataFormat() { public void setupTable() throws Exception { this.table = create(SCHEMA, PartitionSpec.unpartitioned()); this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PUFFIN).build(); + this.parquetFileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PARQUET).build(); } @TestTemplate public void testBasicDVs() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + FileWriterFactory writerFactory = newWriterFactory(table.schema()); // add the first data file @@ -100,6 +114,211 @@ public void testBasicDVs() throws IOException { .contains(dataFile1.location()) .contains(dataFile2.location()); assertThat(result.referencesDataFiles()).isTrue(); + + // commit the deletes + commit(result); + + // verify correctness + assertRows(ImmutableList.of(toRow(11, "aaa"), toRow(12, "aaa"))); + } + + @TestTemplate + public void testRewriteDVs() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + + FileWriterFactory writerFactory = newWriterFactory(table.schema()); + + // add a data file with 3 data records + List rows = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "aaa")); + DataFile dataFile = writeData(writerFactory, parquetFileFactory, rows, table.spec(), null); + table.newFastAppend().appendFile(dataFile).commit(); + + // write the first DV + DVFileWriter dvWriter1 = + new BaseDVFileWriter(fileFactory, new PreviousDeleteLoader(table, ImmutableMap.of())); + dvWriter1.delete(dataFile.location(), 1L, table.spec(), null); + dvWriter1.close(); + + // validate the writer result + DeleteWriteResult result1 = dvWriter1.result(); + assertThat(result1.deleteFiles()).hasSize(1); + assertThat(result1.referencedDataFiles()).containsOnly(dataFile.location()); + assertThat(result1.referencesDataFiles()).isTrue(); + assertThat(result1.rewrittenDeleteFiles()).isEmpty(); + + // commit the first DV + commit(result1); + assertThat(table.currentSnapshot().addedDeleteFiles(table.io())).hasSize(1); + assertThat(table.currentSnapshot().removedDeleteFiles(table.io())).isEmpty(); + + // verify correctness after committing the first DV + assertRows(ImmutableList.of(toRow(1, "aaa"), toRow(3, "aaa"))); + + // write the second DV, merging with the first one + DeleteFile dv1 = Iterables.getOnlyElement(result1.deleteFiles()); + DVFileWriter dvWriter2 = + new BaseDVFileWriter( + fileFactory, + new PreviousDeleteLoader(table, ImmutableMap.of(dataFile.location(), dv1))); + dvWriter2.delete(dataFile.location(), 2L, table.spec(), null); + dvWriter2.close(); + + // validate the writer result + DeleteWriteResult result2 = dvWriter2.result(); + assertThat(result2.deleteFiles()).hasSize(1); + assertThat(result2.referencedDataFiles()).containsOnly(dataFile.location()); + assertThat(result2.referencesDataFiles()).isTrue(); + assertThat(result2.rewrittenDeleteFiles()).hasSize(1); + + // replace DVs + commit(result2); + assertThat(table.currentSnapshot().addedDeleteFiles(table.io())).hasSize(1); + assertThat(table.currentSnapshot().removedDeleteFiles(table.io())).hasSize(1); + + // verify correctness after replacing DVs + assertRows(ImmutableList.of(toRow(1, "aaa"))); + } + + @TestTemplate + public void testRewriteFileScopedPositionDeletes() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + + FileWriterFactory writerFactory = newWriterFactory(table.schema()); + + // add a data file with 3 records + List rows = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "aaa")); + DataFile dataFile = writeData(writerFactory, parquetFileFactory, rows, table.spec(), null); + table.newFastAppend().appendFile(dataFile).commit(); + + // add a file-scoped position delete file + DeleteFile deleteFile = + writePositionDeletes(writerFactory, ImmutableList.of(Pair.of(dataFile.location(), 0L))); + table.newRowDelta().addDeletes(deleteFile).commit(); + + // verify correctness after adding the file-scoped position delete + assertRows(ImmutableList.of(toRow(2, "aaa"), toRow(3, "aaa"))); + + // upgrade the table to V3 to enable DVs + table.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); + + // write a DV, merging with the file-scoped position delete + DVFileWriter dvWriter = + new BaseDVFileWriter( + fileFactory, + new PreviousDeleteLoader(table, ImmutableMap.of(dataFile.location(), deleteFile))); + dvWriter.delete(dataFile.location(), 1L, table.spec(), null); + dvWriter.close(); + + // validate the writer result + DeleteWriteResult result = dvWriter.result(); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.referencedDataFiles()).containsOnly(dataFile.location()); + assertThat(result.referencesDataFiles()).isTrue(); + assertThat(result.rewrittenDeleteFiles()).hasSize(1); + + // replace the position delete file with the DV + commit(result); + assertThat(table.currentSnapshot().addedDeleteFiles(table.io())).hasSize(1); + assertThat(table.currentSnapshot().removedDeleteFiles(table.io())).hasSize(1); + + // verify correctness + assertRows(ImmutableList.of(toRow(3, "aaa"))); + } + + @TestTemplate + public void testApplyPartitionScopedPositionDeletes() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + + FileWriterFactory writerFactory = newWriterFactory(table.schema()); + + // add the first data file with 3 records + List rows1 = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "aaa")); + DataFile dataFile1 = writeData(writerFactory, parquetFileFactory, rows1, table.spec(), null); + table.newFastAppend().appendFile(dataFile1).commit(); + + // add the second data file with 3 records + List rows2 = ImmutableList.of(toRow(4, "aaa"), toRow(5, "aaa"), toRow(6, "aaa")); + DataFile dataFile2 = writeData(writerFactory, parquetFileFactory, rows2, table.spec(), null); + table.newFastAppend().appendFile(dataFile2).commit(); + + // add a position delete file with deletes for both data files + DeleteFile deleteFile = + writePositionDeletes( + writerFactory, + ImmutableList.of( + Pair.of(dataFile1.location(), 0L), + Pair.of(dataFile1.location(), 1L), + Pair.of(dataFile2.location(), 0L))); + table.newRowDelta().addDeletes(deleteFile).commit(); + + // verify correctness with the position delete file + assertRows(ImmutableList.of(toRow(3, "aaa"), toRow(5, "aaa"), toRow(6, "aaa"))); + + // upgrade the table to V3 to enable DVs + table.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); + + // write a DV, applying old positions but keeping the position delete file in place + DVFileWriter dvWriter = + new BaseDVFileWriter( + fileFactory, + new PreviousDeleteLoader(table, ImmutableMap.of(dataFile2.location(), deleteFile))); + dvWriter.delete(dataFile2.location(), 1L, table.spec(), null); + dvWriter.close(); + + // validate the writer result + DeleteWriteResult result = dvWriter.result(); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.referencedDataFiles()).containsOnly(dataFile2.location()); + assertThat(result.referencesDataFiles()).isTrue(); + assertThat(result.rewrittenDeleteFiles()).isEmpty(); + DeleteFile dv = Iterables.getOnlyElement(result.deleteFiles()); + + // commit the DV, ensuring the position delete file remains + commit(result); + assertThat(table.currentSnapshot().addedDeleteFiles(table.io())).hasSize(1); + assertThat(table.currentSnapshot().removedDeleteFiles(table.io())).isEmpty(); + + // verify correctness with DVs and position delete files + assertRows(ImmutableList.of(toRow(3, "aaa"), toRow(6, "aaa"))); + + // verify the position delete file applies only to the data file without the DV + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask task : tasks) { + DeleteFile taskDeleteFile = Iterables.getOnlyElement(task.deletes()); + if (task.file().location().equals(dataFile1.location())) { + assertThat(taskDeleteFile.location()).isEqualTo(deleteFile.location()); + } else { + assertThat(taskDeleteFile.location()).isEqualTo(dv.location()); + } + } + } + } + + private void commit(DeleteWriteResult result) { + RowDelta rowDelta = table.newRowDelta(); + result.rewrittenDeleteFiles().forEach(rowDelta::removeDeletes); + result.deleteFiles().forEach(rowDelta::addDeletes); + rowDelta.commit(); + } + + private void assertRows(Iterable expectedRows) throws IOException { + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); + } + + private DeleteFile writePositionDeletes( + FileWriterFactory writerFactory, List> deletes) throws IOException { + EncryptedOutputFile file = parquetFileFactory.newOutputFile(table.spec(), null); + PositionDeleteWriter writer = + writerFactory.newPositionDeleteWriter(file, table.spec(), null); + PositionDelete posDelete = PositionDelete.create(); + + try (PositionDeleteWriter closableWriter = writer) { + for (Pair delete : deletes) { + closableWriter.write(posDelete.set(delete.first(), delete.second())); + } + } + + return writer.toDeleteFile(); } private static class PreviousDeleteLoader implements Function { From dda62154e69a27da1fa9e6cce59413f988a0e99b Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 13:05:42 -0600 Subject: [PATCH 116/313] Infra: Update DOAP.RDF for Apache Iceberg 1.7.0 (#11492) --- doap.rdf | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doap.rdf b/doap.rdf index 51896c6a8037..4b7418534644 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ - 1.6.1 - 2024-08-27 - 1.6.1 + 1.7.0 + 2024-11-08 + 1.7.0 From 6a16340947ec7f596133104d7945555aabdeee86 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 13:34:08 -0600 Subject: [PATCH 117/313] Docs: Site Update for 1.7.0 Release (#11494) * Docs: Site Update for 1.7.0 Release * Docs: Adds 1.7.0 Release Notes * Update site/docs/releases.md Co-authored-by: Fokko Driesprong --------- Co-authored-by: Fokko Driesprong --- site/docs/releases.md | 82 ++++++++++++++++++++++++++++++++++++++++++- site/mkdocs.yml | 2 +- site/nav.yml | 1 + 3 files changed, 83 insertions(+), 2 deletions(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 6b48e31a0728..2e37d5551459 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -72,10 +72,90 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: ``` +### 1.7.0 release + +The 1.7.0 release contains fixes, dependency updates, and new features. For full release notes please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.7.0). An abridged list follows + +* Deprecation / End of Support + * Java 8 + * Apache Pig +* API + - Add SupportsRecoveryOperations mixin for FileIO [\#10711](https://github.com/apache/iceberg/pull/10711)) + - Add default value APIs and Avro implementation [\#9502](https://github.com/apache/iceberg/pull/9502)) + - Add compatibility checks for Schemas with default values [\#11434](https://github.com/apache/iceberg/pull/11434)) + - Implement types timestamp_ns and timestamptz_ns [\#9008](https://github.com/apache/iceberg/pull/9008)) + - Add addNonDefaultSpec to UpdatePartitionSpec to not set the new partition spec as default [\#10736](https://github.com/apache/iceberg/pull/10736)) + * AWS + - Don't complete multipart upload on finalize for S3OutputStream [\#10874](https://github.com/apache/iceberg/pull/10874)) + - Implement SupportsRecoveryOperations for S3FileIO [\#10721](https://github.com/apache/iceberg/pull/10721)) + - Refresh vended credentials [\#11389](https://github.com/apache/iceberg/pull/11389)) + - Support S3 directory bucket listing [\#11021](https://github.com/apache/iceberg/pull/11021)) + - S3FileIO - Add Cross-Region Bucket Access [\#11259](https://github.com/apache/iceberg/pull/11259)) +* Build + - Build for Java 11 [\#10849](https://github.com/apache/iceberg/pull/10849)) - Removal of Java 8 + - Build: Add checkstyle rule to ban assert usage [\#10886](https://github.com/apache/iceberg/pull/10886)) + - Support building with Java 21 [\#10474](https://github.com/apache/iceberg/pull/10474)) +* Dependencies + - AWS SDK 2.29.1 + - Apache Avro to 1.12.0 + - Spark 3.4 to 3.4.4 + - Spark 3.5 to 3.5.2 + - Netty to 4.1.114.Final + - Jetty to 11.0.24 + - Kafka to 3.8.0 + - Nessie to 0.99.0 + - ORC to 1.9.4 + - Roaring Bitmap to 1.3.0 + - Spring to 5.3.39 + - Sqllite JDBC to 3.46.0.0 + - Hadoop to 3.4.1 +* Core + - Remove dangling deletes as part of RewriteDataFilesAction [\#9724](https://github.com/apache/iceberg/pull/9724)) + - Add a util to compute partition stats [\#11146](https://github.com/apache/iceberg/pull/11146)) + - Add estimateRowCount for Files and Entries Metadata Tables [\#10759](https://github.com/apache/iceberg/pull/10759)) + - Add portable Roaring bitmap for row positions [\#11372](https://github.com/apache/iceberg/pull/11372)) + - Add rewritten delete files to write results [\#11203](https://github.com/apache/iceberg/pull/11203)) + - Add Basic Classes for Iceberg Table Version 3 [\#10760](https://github.com/apache/iceberg/pull/10760)) + - Deprecate ContentCache.invalidateAll [\#10494](https://github.com/apache/iceberg/pull/10494)) + - Deprecate legacy ways for loading position deletes [\#11242](https://github.com/apache/iceberg/pull/11242)) + - Parallelize manifest writing for many new files [\#11086](https://github.com/apache/iceberg/pull/11086)) + - Support appending files with different specs [\#9860](https://github.com/apache/iceberg/pull/9860)) +* Flink + - Introduces the new IcebergSink based on the new V2 Flink Sink Abstraction [\#10179](https://github.com/apache/iceberg/pull/10179)) + - Update Flink to use planned Avro reads [\#11386](https://github.com/apache/iceberg/pull/11386)) + - Infer source parallelism for FLIP-27 source in batch execution mode [\#10832](https://github.com/apache/iceberg/pull/10832)) + - Make FLIP-27 default in SQL and mark the old FlinkSource as deprecated [\#11345](https://github.com/apache/iceberg/pull/11345)) + - Support limit pushdown in FLIP-27 source [\#10748](https://github.com/apache/iceberg/pull/10748)) +* GCS + - Refresh vended credentials [\#11282](https://github.com/apache/iceberg/pull/11282)) +* Hive + - Add View support for HIVE catalog [\#9852](https://github.com/apache/iceberg/pull/9852)) +* OpenAPI + - Add RemovePartitionSpecsUpdate REST update type [\#10846](https://github.com/apache/iceberg/pull/10846)) + - Add endpoint to retrieve valid credentials for a given table [\#11281](https://github.com/apache/iceberg/pull/11281)) + - Standardize credentials in loadTable/loadView responses [\#10722](https://github.com/apache/iceberg/pull/10722)) + - Add Scan Planning Endpoints to open api spec [\#9695](https://github.com/apache/iceberg/pull/9695)) + - Add REST Compatibility Kit [\#10908](https://github.com/apache/iceberg/pull/10908)) +* Spark + - Parallelize reading files in migrate procedures [\#11043](https://github.com/apache/iceberg/pull/11043)) + - Action to compute table stats [\#11106](https://github.com/apache/iceberg/pull/11106)) + - Action to remove dangling deletes [\#11377](https://github.com/apache/iceberg/pull/11377)) + - Add utility to load table state reliably [\#11115](https://github.com/apache/iceberg/pull/11115)) + - Don't change table distribution when only altering local order [\#10774](https://github.com/apache/iceberg/pull/10774)) + - Update Spark to use planned Avro reads [\#11299](https://github.com/apache/iceberg/pull/11299)) + - Spark Action to Analyze table [\#10288](https://github.com/apache/iceberg/pull/10288)) + - Support Column Stats [\#10659](https://github.com/apache/iceberg/pull/10659)) + - Add RewriteTablePath action interface [\#10920](https://github.com/apache/iceberg/pull/10920)) +* Spec + - Add v3 types and type promotion [\#10955](https://github.com/apache/iceberg/pull/10955)) + - Adds Row Lineage [\#11130](https://github.com/apache/iceberg/pull/11130)) + - Deprecate the file system table scheme. [\#10833](https://github.com/apache/iceberg/pull/10833)) + + ### 1.6.0 release Apache Iceberg 1.6.0 was released on July 23, 2024. -The 1.6.0 release contains fixes, dependency updates, and new features (like Kafak Connect commit coordinator and record converters). +The 1.6.0 release contains fixes, dependency updates, and new features (like Kafka Connect commit coordinator and record converters). * Build - Upgrade to Gradle 8.9 ([\#10686](https://github.com/apache/iceberg/pull/10686)) diff --git a/site/mkdocs.yml b/site/mkdocs.yml index db9bafb00f05..646c87392826 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -78,7 +78,7 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.6.1' + icebergVersion: '1.7.0' nessieVersion: '0.92.1' flinkVersion: '1.19.0' flinkVersionMajor: '1.19' diff --git a/site/nav.yml b/site/nav.yml index c13b7cfaa6b1..e54bb6260a77 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -23,6 +23,7 @@ nav: - Docs: - nightly: '!include docs/docs/nightly/mkdocs.yml' - latest: '!include docs/docs/latest/mkdocs.yml' + - 1.7.0: '!include docs/docs/1.7.0/mkdocs.yml' - 1.6.1: '!include docs/docs/1.6.1/mkdocs.yml' - 1.6.0: '!include docs/docs/1.6.0/mkdocs.yml' - 1.5.2: '!include docs/docs/1.5.2/mkdocs.yml' From bbc0d9aad515dc7c9c38e5f37b6d6fa521e7eab3 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 13:34:56 -0600 Subject: [PATCH 118/313] Infra: Add 1.7.0 to issue template (#11491) * Infra: Add 1.7.0 to issue template * Update .github/ISSUE_TEMPLATE/iceberg_bug_report.yml * Missing `)` Co-authored-by: Amogh Jahagirdar --------- Co-authored-by: Fokko Driesprong Co-authored-by: Amogh Jahagirdar --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index a04263e8c943..79b492fff7aa 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -28,7 +28,8 @@ body: description: What Apache Iceberg version are you using? multiple: false options: - - "1.6.1 (latest release)" + - "1.7.0 (latest release)" + - "1.6.1" - "1.6.0" - "1.5.2" - "1.5.1" From df0917d5536eae151d667a62231c0023dcd5f1e9 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 13:35:12 -0600 Subject: [PATCH 119/313] Build: Let revapi compare against 1.7.0 (#11490) --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index d5bb33b7ff25..7990ffbadcb2 100644 --- a/build.gradle +++ b/build.gradle @@ -135,7 +135,7 @@ subprojects { revapi { oldGroup = project.group oldName = project.name - oldVersion = "1.6.0" + oldVersion = "1.7.0" } tasks.register('showDeprecationRulesOnRevApiFailure') { From cd25937dc7438637d11cd54d2065d391bdc6d405 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 14:46:15 -0600 Subject: [PATCH 120/313] Docs: Adds Release notes for 1.6.1 (#11500) --- site/docs/releases.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/site/docs/releases.md b/site/docs/releases.md index 2e37d5551459..22456416da19 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -151,6 +151,18 @@ The 1.7.0 release contains fixes, dependency updates, and new features. For full - Adds Row Lineage [\#11130](https://github.com/apache/iceberg/pull/11130)) - Deprecate the file system table scheme. [\#10833](https://github.com/apache/iceberg/pull/10833)) +### 1.6.1 release + +Apache Iceberg 1.6.1 was released on August 27, 2024. + +The 1.6.1 Release contains bug fixes and performance improvements. For full release notes visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.6.1) + +* Core + - Limit ParallelIterable memory consumption by yielding in tasks ([\#10787](https://github.com/apache/iceberg/#10787)) + - Drop ParallelIterable's queue low water mark ([\#10979](https://github.com/apache/iceberg/#10979)) +* Dependencies + - ORC 1.9.4 + ### 1.6.0 release Apache Iceberg 1.6.0 was released on July 23, 2024. From dfee4cb35e2d18ff22625d360ed8c4af499e694d Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 14:47:21 -0600 Subject: [PATCH 121/313] Docs: Fixes Release Formatting for 1.7.0 Release Notes (#11499) --- site/docs/releases.md | 100 +++++++++++++++++++++--------------------- 1 file changed, 50 insertions(+), 50 deletions(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 22456416da19..e90887282514 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -77,24 +77,24 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: The 1.7.0 release contains fixes, dependency updates, and new features. For full release notes please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.7.0). An abridged list follows * Deprecation / End of Support - * Java 8 - * Apache Pig + - Java 8 + - Apache Pig * API - - Add SupportsRecoveryOperations mixin for FileIO [\#10711](https://github.com/apache/iceberg/pull/10711)) - - Add default value APIs and Avro implementation [\#9502](https://github.com/apache/iceberg/pull/9502)) - - Add compatibility checks for Schemas with default values [\#11434](https://github.com/apache/iceberg/pull/11434)) - - Implement types timestamp_ns and timestamptz_ns [\#9008](https://github.com/apache/iceberg/pull/9008)) - - Add addNonDefaultSpec to UpdatePartitionSpec to not set the new partition spec as default [\#10736](https://github.com/apache/iceberg/pull/10736)) - * AWS - - Don't complete multipart upload on finalize for S3OutputStream [\#10874](https://github.com/apache/iceberg/pull/10874)) - - Implement SupportsRecoveryOperations for S3FileIO [\#10721](https://github.com/apache/iceberg/pull/10721)) - - Refresh vended credentials [\#11389](https://github.com/apache/iceberg/pull/11389)) - - Support S3 directory bucket listing [\#11021](https://github.com/apache/iceberg/pull/11021)) - - S3FileIO - Add Cross-Region Bucket Access [\#11259](https://github.com/apache/iceberg/pull/11259)) + - Add SupportsRecoveryOperations mixin for FileIO ([\#10711](https://github.com/apache/iceberg/pull/10711)) + - Add default value APIs and Avro implementation ([\#9502](https://github.com/apache/iceberg/pull/9502)) + - Add compatibility checks for Schemas with default values ([\#11434](https://github.com/apache/iceberg/pull/11434)) + - Implement types timestamp_ns and timestamptz_ns ([\#9008](https://github.com/apache/iceberg/pull/9008)) + - Add addNonDefaultSpec to UpdatePartitionSpec to not set the new partition spec as default ([\#10736](https://github.com/apache/iceberg/pull/10736)) +* AWS + - Don't complete multipart upload on finalize for S3OutputStream ([\#10874](https://github.com/apache/iceberg/pull/10874)) + - Implement SupportsRecoveryOperations for S3FileIO ([\#10721](https://github.com/apache/iceberg/pull/10721)) + - Refresh vended credentials ([\#11389](https://github.com/apache/iceberg/pull/11389)) + - Support S3 directory bucket listing ([\#11021](https://github.com/apache/iceberg/pull/11021)) + - S3FileIO - Add Cross-Region Bucket Access ([\#11259](https://github.com/apache/iceberg/pull/11259)) * Build - - Build for Java 11 [\#10849](https://github.com/apache/iceberg/pull/10849)) - Removal of Java 8 - - Build: Add checkstyle rule to ban assert usage [\#10886](https://github.com/apache/iceberg/pull/10886)) - - Support building with Java 21 [\#10474](https://github.com/apache/iceberg/pull/10474)) + - Build for Java 11 ([\#10849](https://github.com/apache/iceberg/pull/10849)) - Removal of Java 8 + - Build: Add checkstyle rule to ban assert usage ([\#10886](https://github.com/apache/iceberg/pull/10886)) + - Support building with Java 21 ([\#10474](https://github.com/apache/iceberg/pull/10474)) * Dependencies - AWS SDK 2.29.1 - Apache Avro to 1.12.0 @@ -110,46 +110,46 @@ The 1.7.0 release contains fixes, dependency updates, and new features. For full - Sqllite JDBC to 3.46.0.0 - Hadoop to 3.4.1 * Core - - Remove dangling deletes as part of RewriteDataFilesAction [\#9724](https://github.com/apache/iceberg/pull/9724)) - - Add a util to compute partition stats [\#11146](https://github.com/apache/iceberg/pull/11146)) - - Add estimateRowCount for Files and Entries Metadata Tables [\#10759](https://github.com/apache/iceberg/pull/10759)) - - Add portable Roaring bitmap for row positions [\#11372](https://github.com/apache/iceberg/pull/11372)) - - Add rewritten delete files to write results [\#11203](https://github.com/apache/iceberg/pull/11203)) - - Add Basic Classes for Iceberg Table Version 3 [\#10760](https://github.com/apache/iceberg/pull/10760)) - - Deprecate ContentCache.invalidateAll [\#10494](https://github.com/apache/iceberg/pull/10494)) - - Deprecate legacy ways for loading position deletes [\#11242](https://github.com/apache/iceberg/pull/11242)) - - Parallelize manifest writing for many new files [\#11086](https://github.com/apache/iceberg/pull/11086)) - - Support appending files with different specs [\#9860](https://github.com/apache/iceberg/pull/9860)) + - Remove dangling deletes as part of RewriteDataFilesAction ([\#9724](https://github.com/apache/iceberg/pull/9724)) + - Add a util to compute partition stats ([\#11146](https://github.com/apache/iceberg/pull/11146)) + - Add estimateRowCount for Files and Entries Metadata Tables ([\#10759](https://github.com/apache/iceberg/pull/10759)) + - Add portable Roaring bitmap for row positions ([\#11372](https://github.com/apache/iceberg/pull/11372)) + - Add rewritten delete files to write results ([\#11203](https://github.com/apache/iceberg/pull/11203)) + - Add Basic Classes for Iceberg Table Version 3 ([\#10760](https://github.com/apache/iceberg/pull/10760)) + - Deprecate ContentCache.invalidateAll ([\#10494](https://github.com/apache/iceberg/pull/10494)) + - Deprecate legacy ways for loading position deletes ([\#11242](https://github.com/apache/iceberg/pull/11242)) + - Parallelize manifest writing for many new files ([\#11086](https://github.com/apache/iceberg/pull/11086)) + - Support appending files with different specs ([\#9860](https://github.com/apache/iceberg/pull/9860)) * Flink - - Introduces the new IcebergSink based on the new V2 Flink Sink Abstraction [\#10179](https://github.com/apache/iceberg/pull/10179)) - - Update Flink to use planned Avro reads [\#11386](https://github.com/apache/iceberg/pull/11386)) - - Infer source parallelism for FLIP-27 source in batch execution mode [\#10832](https://github.com/apache/iceberg/pull/10832)) - - Make FLIP-27 default in SQL and mark the old FlinkSource as deprecated [\#11345](https://github.com/apache/iceberg/pull/11345)) - - Support limit pushdown in FLIP-27 source [\#10748](https://github.com/apache/iceberg/pull/10748)) + - Introduces the new IcebergSink based on the new V2 Flink Sink Abstraction ([\#10179](https://github.com/apache/iceberg/pull/10179)) + - Update Flink to use planned Avro reads ([\#11386](https://github.com/apache/iceberg/pull/11386)) + - Infer source parallelism for FLIP-27 source in batch execution mode ([\#10832](https://github.com/apache/iceberg/pull/10832)) + - Make FLIP-27 default in SQL and mark the old FlinkSource as deprecated ([\#11345](https://github.com/apache/iceberg/pull/11345)) + - Support limit pushdown in FLIP-27 source ([\#10748](https://github.com/apache/iceberg/pull/10748)) * GCS - - Refresh vended credentials [\#11282](https://github.com/apache/iceberg/pull/11282)) + - Refresh vended credentials ([\#11282](https://github.com/apache/iceberg/pull/11282)) * Hive - - Add View support for HIVE catalog [\#9852](https://github.com/apache/iceberg/pull/9852)) + - Add View support for HIVE catalog ([\#9852](https://github.com/apache/iceberg/pull/9852)) * OpenAPI - - Add RemovePartitionSpecsUpdate REST update type [\#10846](https://github.com/apache/iceberg/pull/10846)) - - Add endpoint to retrieve valid credentials for a given table [\#11281](https://github.com/apache/iceberg/pull/11281)) - - Standardize credentials in loadTable/loadView responses [\#10722](https://github.com/apache/iceberg/pull/10722)) - - Add Scan Planning Endpoints to open api spec [\#9695](https://github.com/apache/iceberg/pull/9695)) - - Add REST Compatibility Kit [\#10908](https://github.com/apache/iceberg/pull/10908)) + - Add RemovePartitionSpecsUpdate REST update type ([\#10846](https://github.com/apache/iceberg/pull/10846)) + - Add endpoint to retrieve valid credentials for a given table ([\#11281](https://github.com/apache/iceberg/pull/11281)) + - Standardize credentials in loadTable/loadView responses ([\#10722](https://github.com/apache/iceberg/pull/10722)) + - Add Scan Planning Endpoints to open api spec ([\#9695](https://github.com/apache/iceberg/pull/9695)) + - Add REST Compatibility Kit ([\#10908](https://github.com/apache/iceberg/pull/10908)) * Spark - - Parallelize reading files in migrate procedures [\#11043](https://github.com/apache/iceberg/pull/11043)) - - Action to compute table stats [\#11106](https://github.com/apache/iceberg/pull/11106)) - - Action to remove dangling deletes [\#11377](https://github.com/apache/iceberg/pull/11377)) - - Add utility to load table state reliably [\#11115](https://github.com/apache/iceberg/pull/11115)) - - Don't change table distribution when only altering local order [\#10774](https://github.com/apache/iceberg/pull/10774)) - - Update Spark to use planned Avro reads [\#11299](https://github.com/apache/iceberg/pull/11299)) - - Spark Action to Analyze table [\#10288](https://github.com/apache/iceberg/pull/10288)) - - Support Column Stats [\#10659](https://github.com/apache/iceberg/pull/10659)) - - Add RewriteTablePath action interface [\#10920](https://github.com/apache/iceberg/pull/10920)) + - Parallelize reading files in migrate procedures ([\#11043](https://github.com/apache/iceberg/pull/11043)) + - Action to compute table stats ([\#11106](https://github.com/apache/iceberg/pull/11106)) + - Action to remove dangling deletes ([\#11377](https://github.com/apache/iceberg/pull/11377)) + - Add utility to load table state reliably ([\#11115](https://github.com/apache/iceberg/pull/11115)) + - Don't change table distribution when only altering local order ([\#10774](https://github.com/apache/iceberg/pull/10774)) + - Update Spark to use planned Avro reads ([\#11299](https://github.com/apache/iceberg/pull/11299)) + - Spark Action to Analyze table ([\#10288](https://github.com/apache/iceberg/pull/10288)) + - Support Column Stats ([\#10659](https://github.com/apache/iceberg/pull/10659)) + - Add RewriteTablePath action interface ([\#10920](https://github.com/apache/iceberg/pull/10920)) * Spec - - Add v3 types and type promotion [\#10955](https://github.com/apache/iceberg/pull/10955)) - - Adds Row Lineage [\#11130](https://github.com/apache/iceberg/pull/11130)) - - Deprecate the file system table scheme. [\#10833](https://github.com/apache/iceberg/pull/10833)) + - Add v3 types and type promotion ([\#10955](https://github.com/apache/iceberg/pull/10955)) + - Adds Row Lineage ([\#11130](https://github.com/apache/iceberg/pull/11130)) + - Deprecate the file system table scheme. ([\#10833](https://github.com/apache/iceberg/pull/10833)) ### 1.6.1 release From 82a2362afce28bb2c8a390aadf6babca2d731822 Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+sungwy@users.noreply.github.com> Date: Fri, 8 Nov 2024 16:39:30 -0500 Subject: [PATCH 122/313] DOCS: Explicitly specify `operation` as a _required_ field of `summary` field (#11355) --- format/spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index bdc328451cf2..c9a5f715102d 100644 --- a/format/spec.md +++ b/format/spec.md @@ -661,7 +661,7 @@ A snapshot consists of the following fields: | _required_ | _required_ | _required_ | **`timestamp-ms`** | A timestamp when the snapshot was created, used for garbage collection and table inspection | | _optional_ | _required_ | _required_ | **`manifest-list`** | The location of a manifest list for this snapshot that tracks manifest files with additional metadata | | _optional_ | | | **`manifests`** | A list of manifest file locations. Must be omitted if `manifest-list` is present | -| _optional_ | _required_ | _required_ | **`summary`** | A string map that summarizes the snapshot changes, including `operation` (see below) | +| _optional_ | _required_ | _required_ | **`summary`** | A string map that summarizes the snapshot changes, including `operation` as a _required_ field (see below) | | _optional_ | _optional_ | _optional_ | **`schema-id`** | ID of the table's current schema when the snapshot was created | | | | _optional_ | **`first-row-id`** | The first `_row_id` assigned to the first row in the first data file in the first manifest, see [Row Lineage](#row-lineage) | From 1c576c5952fbc623591e800408cdba1518e6a410 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 8 Nov 2024 15:57:37 -0800 Subject: [PATCH 123/313] Spark: Exclude reading _pos column if it's not in the scan list (#11390) --- .../org/apache/iceberg/data/DeleteFilter.java | 20 +++++++++++++++---- .../iceberg/spark/source/BaseBatchReader.java | 17 +++++++++++++++- .../iceberg/spark/source/BaseReader.java | 5 +++-- .../iceberg/spark/source/BatchDataReader.java | 2 +- .../spark/source/ChangelogRowReader.java | 5 +++-- .../spark/source/EqualityDeleteRowReader.java | 2 +- .../iceberg/spark/source/RowDataReader.java | 3 ++- .../iceberg/spark/source/BaseBatchReader.java | 17 +++++++++++++++- .../iceberg/spark/source/BaseReader.java | 5 +++-- .../iceberg/spark/source/BatchDataReader.java | 2 +- .../spark/source/ChangelogRowReader.java | 5 +++-- .../spark/source/EqualityDeleteRowReader.java | 2 +- .../iceberg/spark/source/RowDataReader.java | 3 ++- .../iceberg/spark/source/BaseBatchReader.java | 17 +++++++++++++++- .../iceberg/spark/source/BaseReader.java | 5 +++-- .../iceberg/spark/source/BatchDataReader.java | 2 +- .../spark/source/ChangelogRowReader.java | 5 +++-- .../spark/source/EqualityDeleteRowReader.java | 2 +- .../iceberg/spark/source/RowDataReader.java | 3 ++- 19 files changed, 94 insertions(+), 28 deletions(-) diff --git a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java index e7d8445cf8c8..aa5e00fd0ef4 100644 --- a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java +++ b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java @@ -69,7 +69,8 @@ protected DeleteFilter( List deletes, Schema tableSchema, Schema requestedSchema, - DeleteCounter counter) { + DeleteCounter counter, + boolean needRowPosCol) { this.filePath = filePath; this.counter = counter; @@ -93,13 +94,23 @@ protected DeleteFilter( this.posDeletes = posDeleteBuilder.build(); this.eqDeletes = eqDeleteBuilder.build(); - this.requiredSchema = fileProjection(tableSchema, requestedSchema, posDeletes, eqDeletes); + this.requiredSchema = + fileProjection(tableSchema, requestedSchema, posDeletes, eqDeletes, needRowPosCol); this.posAccessor = requiredSchema.accessorForField(MetadataColumns.ROW_POSITION.fieldId()); this.hasIsDeletedColumn = requiredSchema.findField(MetadataColumns.IS_DELETED.fieldId()) != null; this.isDeletedColumnPosition = requiredSchema.columns().indexOf(MetadataColumns.IS_DELETED); } + protected DeleteFilter( + String filePath, + List deletes, + Schema tableSchema, + Schema requestedSchema, + DeleteCounter counter) { + this(filePath, deletes, tableSchema, requestedSchema, counter, true); + } + protected DeleteFilter( String filePath, List deletes, Schema tableSchema, Schema requestedSchema) { this(filePath, deletes, tableSchema, requestedSchema, new DeleteCounter()); @@ -251,13 +262,14 @@ private static Schema fileProjection( Schema tableSchema, Schema requestedSchema, List posDeletes, - List eqDeletes) { + List eqDeletes, + boolean needRowPosCol) { if (posDeletes.isEmpty() && eqDeletes.isEmpty()) { return requestedSchema; } Set requiredIds = Sets.newLinkedHashSet(); - if (!posDeletes.isEmpty()) { + if (needRowPosCol && !posDeletes.isEmpty()) { requiredIds.add(MetadataColumns.ROW_POSITION.fieldId()); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index c05b694a60dc..49c43952135c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.source; +import java.util.List; import java.util.Map; import java.util.Set; import org.apache.iceberg.FileFormat; @@ -31,10 +32,12 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.vectorized.ColumnarBatch; abstract class BaseBatchReader extends BaseReader { @@ -81,9 +84,21 @@ private CloseableIterable newParquetIterable( SparkDeleteFilter deleteFilter) { // get required schema if there are deletes Schema requiredSchema = deleteFilter != null ? deleteFilter.requiredSchema() : expectedSchema(); + boolean hasPositionDelete = deleteFilter != null ? deleteFilter.hasPosDeletes() : false; + Schema projectedSchema = requiredSchema; + if (hasPositionDelete) { + // We need to add MetadataColumns.ROW_POSITION in the schema for + // ReadConf.generateOffsetToStartPos(Schema schema). This is not needed any + // more after #10107 is merged. + List columns = Lists.newArrayList(requiredSchema.columns()); + if (!columns.contains(MetadataColumns.ROW_POSITION)) { + columns.add(MetadataColumns.ROW_POSITION); + projectedSchema = new Schema(columns); + } + } return Parquet.read(inputFile) - .project(requiredSchema) + .project(projectedSchema) .split(start, length) .createBatchedReaderFunc( fileSchema -> diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index 0882edcb7c4a..96ff430c179d 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -257,8 +257,9 @@ protected static Object convertConstant(Type type, Object value) { protected class SparkDeleteFilter extends DeleteFilter { private final InternalRowWrapper asStructLike; - SparkDeleteFilter(String filePath, List deletes, DeleteCounter counter) { - super(filePath, deletes, tableSchema, expectedSchema, counter); + SparkDeleteFilter( + String filePath, List deletes, DeleteCounter counter, boolean needRowPosCol) { + super(filePath, deletes, tableSchema, expectedSchema, counter, needRowPosCol); this.asStructLike = new InternalRowWrapper( SparkSchemaUtil.convert(requiredSchema()), requiredSchema().asStruct()); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index 389ad1d5a2d9..a2cb74c926c9 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -96,7 +96,7 @@ protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter deleteFilter = task.deletes().isEmpty() ? null - : new SparkDeleteFilter(filePath, task.deletes(), counter()); + : new SparkDeleteFilter(filePath, task.deletes(), counter(), false); return newBatchIterable( inputFile, diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index 572f955884a3..25cd9eda6bce 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -111,13 +111,14 @@ private CloseableIterable openChangelogScanTask(ChangelogScanTask t CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { String filePath = task.file().path().toString(); - SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter()); + SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { String filePath = task.file().path().toString(); - SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.existingDeletes(), counter()); + SparkDeleteFilter deletes = + new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java index f5b98a5a43bd..7aa5a971562a 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java @@ -41,7 +41,7 @@ public EqualityDeleteRowReader( @Override protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter matches = - new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter()); + new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = matches.requiredSchema(); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 9356f62f3593..33b1d6275d6e 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -83,7 +83,8 @@ protected Stream> referencedFiles(FileScanTask task) { protected CloseableIterator open(FileScanTask task) { String filePath = task.file().path().toString(); LOG.debug("Opening data file {}", filePath); - SparkDeleteFilter deleteFilter = new SparkDeleteFilter(filePath, task.deletes(), counter()); + SparkDeleteFilter deleteFilter = + new SparkDeleteFilter(filePath, task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = deleteFilter.requiredSchema(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index c05b694a60dc..49c43952135c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.source; +import java.util.List; import java.util.Map; import java.util.Set; import org.apache.iceberg.FileFormat; @@ -31,10 +32,12 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.vectorized.ColumnarBatch; abstract class BaseBatchReader extends BaseReader { @@ -81,9 +84,21 @@ private CloseableIterable newParquetIterable( SparkDeleteFilter deleteFilter) { // get required schema if there are deletes Schema requiredSchema = deleteFilter != null ? deleteFilter.requiredSchema() : expectedSchema(); + boolean hasPositionDelete = deleteFilter != null ? deleteFilter.hasPosDeletes() : false; + Schema projectedSchema = requiredSchema; + if (hasPositionDelete) { + // We need to add MetadataColumns.ROW_POSITION in the schema for + // ReadConf.generateOffsetToStartPos(Schema schema). This is not needed any + // more after #10107 is merged. + List columns = Lists.newArrayList(requiredSchema.columns()); + if (!columns.contains(MetadataColumns.ROW_POSITION)) { + columns.add(MetadataColumns.ROW_POSITION); + projectedSchema = new Schema(columns); + } + } return Parquet.read(inputFile) - .project(requiredSchema) + .project(projectedSchema) .split(start, length) .createBatchedReaderFunc( fileSchema -> diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index 238069e1c9a5..891640843668 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -262,8 +262,9 @@ protected static Object convertConstant(Type type, Object value) { protected class SparkDeleteFilter extends DeleteFilter { private final InternalRowWrapper asStructLike; - SparkDeleteFilter(String filePath, List deletes, DeleteCounter counter) { - super(filePath, deletes, tableSchema, expectedSchema, counter); + SparkDeleteFilter( + String filePath, List deletes, DeleteCounter counter, boolean needRowPosCol) { + super(filePath, deletes, tableSchema, expectedSchema, counter, needRowPosCol); this.asStructLike = new InternalRowWrapper( SparkSchemaUtil.convert(requiredSchema()), requiredSchema().asStruct()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index 389ad1d5a2d9..a2cb74c926c9 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -96,7 +96,7 @@ protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter deleteFilter = task.deletes().isEmpty() ? null - : new SparkDeleteFilter(filePath, task.deletes(), counter()); + : new SparkDeleteFilter(filePath, task.deletes(), counter(), false); return newBatchIterable( inputFile, diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index 572f955884a3..25cd9eda6bce 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -111,13 +111,14 @@ private CloseableIterable openChangelogScanTask(ChangelogScanTask t CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { String filePath = task.file().path().toString(); - SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter()); + SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { String filePath = task.file().path().toString(); - SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.existingDeletes(), counter()); + SparkDeleteFilter deletes = + new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java index f5b98a5a43bd..7aa5a971562a 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java @@ -41,7 +41,7 @@ public EqualityDeleteRowReader( @Override protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter matches = - new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter()); + new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = matches.requiredSchema(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 9356f62f3593..33b1d6275d6e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -83,7 +83,8 @@ protected Stream> referencedFiles(FileScanTask task) { protected CloseableIterator open(FileScanTask task) { String filePath = task.file().path().toString(); LOG.debug("Opening data file {}", filePath); - SparkDeleteFilter deleteFilter = new SparkDeleteFilter(filePath, task.deletes(), counter()); + SparkDeleteFilter deleteFilter = + new SparkDeleteFilter(filePath, task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = deleteFilter.requiredSchema(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index c05b694a60dc..49c43952135c 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.source; +import java.util.List; import java.util.Map; import java.util.Set; import org.apache.iceberg.FileFormat; @@ -31,10 +32,12 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.vectorized.ColumnarBatch; abstract class BaseBatchReader extends BaseReader { @@ -81,9 +84,21 @@ private CloseableIterable newParquetIterable( SparkDeleteFilter deleteFilter) { // get required schema if there are deletes Schema requiredSchema = deleteFilter != null ? deleteFilter.requiredSchema() : expectedSchema(); + boolean hasPositionDelete = deleteFilter != null ? deleteFilter.hasPosDeletes() : false; + Schema projectedSchema = requiredSchema; + if (hasPositionDelete) { + // We need to add MetadataColumns.ROW_POSITION in the schema for + // ReadConf.generateOffsetToStartPos(Schema schema). This is not needed any + // more after #10107 is merged. + List columns = Lists.newArrayList(requiredSchema.columns()); + if (!columns.contains(MetadataColumns.ROW_POSITION)) { + columns.add(MetadataColumns.ROW_POSITION); + projectedSchema = new Schema(columns); + } + } return Parquet.read(inputFile) - .project(requiredSchema) + .project(projectedSchema) .split(start, length) .createBatchedReaderFunc( fileSchema -> diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index 3c9438480d0d..207035bd30ac 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -249,8 +249,9 @@ protected static Object convertConstant(Type type, Object value) { protected class SparkDeleteFilter extends DeleteFilter { private final InternalRowWrapper asStructLike; - SparkDeleteFilter(String filePath, List deletes, DeleteCounter counter) { - super(filePath, deletes, tableSchema, expectedSchema, counter); + SparkDeleteFilter( + String filePath, List deletes, DeleteCounter counter, boolean needRowPosCol) { + super(filePath, deletes, tableSchema, expectedSchema, counter, needRowPosCol); this.asStructLike = new InternalRowWrapper( SparkSchemaUtil.convert(requiredSchema()), requiredSchema().asStruct()); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index 389ad1d5a2d9..a2cb74c926c9 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -96,7 +96,7 @@ protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter deleteFilter = task.deletes().isEmpty() ? null - : new SparkDeleteFilter(filePath, task.deletes(), counter()); + : new SparkDeleteFilter(filePath, task.deletes(), counter(), false); return newBatchIterable( inputFile, diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index 572f955884a3..25cd9eda6bce 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -111,13 +111,14 @@ private CloseableIterable openChangelogScanTask(ChangelogScanTask t CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { String filePath = task.file().path().toString(); - SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter()); + SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { String filePath = task.file().path().toString(); - SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.existingDeletes(), counter()); + SparkDeleteFilter deletes = + new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java index f5b98a5a43bd..7aa5a971562a 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java @@ -41,7 +41,7 @@ public EqualityDeleteRowReader( @Override protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter matches = - new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter()); + new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = matches.requiredSchema(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 9356f62f3593..33b1d6275d6e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -83,7 +83,8 @@ protected Stream> referencedFiles(FileScanTask task) { protected CloseableIterator open(FileScanTask task) { String filePath = task.file().path().toString(); LOG.debug("Opening data file {}", filePath); - SparkDeleteFilter deleteFilter = new SparkDeleteFilter(filePath, task.deletes(), counter()); + SparkDeleteFilter deleteFilter = + new SparkDeleteFilter(filePath, task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = deleteFilter.requiredSchema(); From ea21a533bc42a11c92d02d7441786bce6838b0b6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 11 Nov 2024 09:10:33 +0100 Subject: [PATCH 124/313] Build: Bump mkdocs-redirects from 1.2.1 to 1.2.2 (#11511) Bumps [mkdocs-redirects](https://github.com/mkdocs/mkdocs-redirects) from 1.2.1 to 1.2.2. - [Release notes](https://github.com/mkdocs/mkdocs-redirects/releases) - [Commits](https://github.com/mkdocs/mkdocs-redirects/compare/v1.2.1...v1.2.2) --- updated-dependencies: - dependency-name: mkdocs-redirects dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index c5cc9261dd78..81b468836491 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -20,4 +20,4 @@ mkdocs-macros-plugin==1.3.7 mkdocs-material==9.5.43 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix -mkdocs-redirects==1.2.1 +mkdocs-redirects==1.2.2 From aa0aeb0cfa77687312fbacee2bfd8ab0b9ba452b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 11 Nov 2024 09:27:40 +0100 Subject: [PATCH 125/313] Build: Bump mkdocs-material from 9.5.43 to 9.5.44 (#11510) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.43 to 9.5.44. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.5.43...9.5.44) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 81b468836491..28056f56c84b 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.3.7 -mkdocs-material==9.5.43 +mkdocs-material==9.5.44 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.2 From 981f1ea2f5d11231139007866831288225d154a5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 11 Nov 2024 14:40:46 +0100 Subject: [PATCH 126/313] Build: Bump software.amazon.awssdk:bom from 2.29.6 to 2.29.9 (#11509) Bumps software.amazon.awssdk:bom from 2.29.6 to 2.29.9. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 091d0a9ec5f9..85fecabcae1a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.29.6" +awssdk-bom = "2.29.9" azuresdk-bom = "1.2.29" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" From d1fd492b865191c40d87795746e058f82fc14026 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 11 Nov 2024 21:41:03 +0800 Subject: [PATCH 127/313] Docs: Update multi-engine support after 1.7.0 release (#11503) --- site/docs/multi-engine-support.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index ad67ba53cda3..ac17c29af546 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -90,10 +90,10 @@ Users should continuously upgrade their Flink version to stay up-to-date. | 1.14 | End of Life | 0.13.0 | 1.2.0 | [iceberg-flink-runtime-1.14](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.14/1.2.0/iceberg-flink-runtime-1.14-1.2.0.jar) | | 1.15 | End of Life | 0.14.0 | 1.4.3 | [iceberg-flink-runtime-1.15](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.15/1.4.3/iceberg-flink-runtime-1.15-1.4.3.jar) | | 1.16 | End of Life | 1.1.0 | 1.5.0 | [iceberg-flink-runtime-1.16](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.16/1.5.0/iceberg-flink-runtime-1.16-1.5.0.jar) | -| 1.17 | Deprecated | 1.3.0 | 1.6.0 | [iceberg-flink-runtime-1.17](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.17/{{ icebergVersion }}/iceberg-flink-runtime-1.17-{{ icebergVersion }}.jar) | +| 1.17 | End of Life | 1.3.0 | 1.6.1 | [iceberg-flink-runtime-1.17](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.17/1.6.1/iceberg-flink-runtime-1.17-1.6.1.jar) | | 1.18 | Maintained | 1.5.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.18](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.18/{{ icebergVersion }}/iceberg-flink-runtime-1.18-{{ icebergVersion }}.jar) | | 1.19 | Maintained | 1.6.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.19](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.19/{{ icebergVersion }}/iceberg-flink-runtime-1.19-{{ icebergVersion }}.jar) | -| 1.20 | Maintained | 1.7.0 (to be released) | - | - | +| 1.20 | Maintained | 1.7.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.20](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.20/{{ icebergVersion }}/iceberg-flink-runtime-1.20-{{ icebergVersion }}.jar) | @@ -105,7 +105,7 @@ Users should continuously upgrade their Flink version to stay up-to-date. | Version | Recommended minor version | Lifecycle Stage | Initial Iceberg Support | Latest Iceberg Support | Latest Runtime Jar | | -------------- | ------------------------- | ----------------- | ----------------------- | ---------------------- | ------------------ | -| 2 | 2.3.8 | Maintained | 0.8.0-incubating | {{ icebergVersion }} | [iceberg-hive-runtime](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-hive-runtime/{{ icebergVersion }}/iceberg-hive-runtime-{{ icebergVersion }}.jar) | +| 2 | 2.3.8 | Deprecated | 0.8.0-incubating | {{ icebergVersion }} | [iceberg-hive-runtime](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-hive-runtime/{{ icebergVersion }}/iceberg-hive-runtime-{{ icebergVersion }}.jar) | | 3 | 3.1.2 | Maintained | 0.10.0 | {{ icebergVersion }} | [iceberg-hive-runtime](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-hive-runtime/{{ icebergVersion }}/iceberg-hive-runtime-{{ icebergVersion }}.jar) | From 5530605ff982e24a9a7894b21b34ae0eddf7dfef Mon Sep 17 00:00:00 2001 From: dongwang Date: Tue, 12 Nov 2024 00:58:41 +0800 Subject: [PATCH 128/313] Spark: Fix typo in Spark ddl comment (#11517) Co-authored-by: hantangwangd --- docs/docs/spark-ddl.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/docs/spark-ddl.md b/docs/docs/spark-ddl.md index 1d1dc695d105..1e71d47bf6ee 100644 --- a/docs/docs/spark-ddl.md +++ b/docs/docs/spark-ddl.md @@ -401,7 +401,7 @@ To set the write order for a table, use `WRITE ORDERED BY`: ```sql ALTER TABLE prod.db.sample WRITE ORDERED BY category, id --- use optional ASC/DEC keyword to specify sort order of each field (default ASC) +-- use optional ASC/DESC keyword to specify sort order of each field (default ASC) ALTER TABLE prod.db.sample WRITE ORDERED BY category ASC, id DESC -- use optional NULLS FIRST/NULLS LAST keyword to specify null order of each field (default FIRST) ALTER TABLE prod.db.sample WRITE ORDERED BY category ASC NULLS LAST, id DESC NULLS FIRST From af3fbfe0786f9016f9cc645a80d0925c7334eb33 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 11 Nov 2024 22:08:23 +0100 Subject: [PATCH 129/313] Core: Support commits with DVs (#11495) --- .../java/org/apache/iceberg/BaseRowDelta.java | 2 + .../iceberg/MergingSnapshotProducer.java | 92 +++++- .../iceberg/DeleteFileIndexTestBase.java | 31 +- .../ScanPlanningAndReportingTestBase.java | 27 +- .../java/org/apache/iceberg/TestBase.java | 47 +++ .../org/apache/iceberg/TestBatchScans.java | 2 +- .../apache/iceberg/TestCommitReporting.java | 30 +- .../iceberg/TestEntriesMetadataTable.java | 2 +- .../iceberg/TestMetadataTableScans.java | 71 ++--- ...adataTableScansWithPartitionEvolution.java | 2 +- .../org/apache/iceberg/TestRewriteFiles.java | 3 + .../apache/iceberg/TestRewriteManifests.java | 44 +-- .../java/org/apache/iceberg/TestRowDelta.java | 270 ++++++++++++------ .../java/org/apache/iceberg/TestSnapshot.java | 10 +- .../apache/iceberg/TestSnapshotSummary.java | 6 +- .../org/apache/iceberg/io/TestDVWriters.java | 5 + .../TestSparkDistributedDataScanDeletes.java | 6 +- ...TestSparkDistributedDataScanReporting.java | 6 +- .../actions/TestRewriteManifestsAction.java | 62 +++- 19 files changed, 506 insertions(+), 212 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index 85c2269ee526..372fc5367f08 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -139,6 +139,8 @@ protected void validate(TableMetadata base, Snapshot parent) { if (validateNewDeleteFiles) { validateNoNewDeleteFiles(base, startingSnapshotId, conflictDetectionFilter, parent); } + + validateAddedDVs(base, startingSnapshotId, conflictDetectionFilter, parent); } } } diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 50885dbb06c7..6198ad00f680 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -48,11 +48,13 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.DataFileSet; import org.apache.iceberg.util.DeleteFileSet; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PartitionSet; import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,6 +72,9 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { // delete files can be added in "overwrite" or "delete" operations private static final Set VALIDATE_ADDED_DELETE_FILES_OPERATIONS = ImmutableSet.of(DataOperations.OVERWRITE, DataOperations.DELETE); + // DVs can be added in "overwrite", "delete", and "replace" operations + private static final Set VALIDATE_ADDED_DVS_OPERATIONS = + ImmutableSet.of(DataOperations.OVERWRITE, DataOperations.DELETE, DataOperations.REPLACE); private final String tableName; private final TableOperations ops; @@ -83,6 +88,7 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final Map newDataFilesBySpec = Maps.newHashMap(); private Long newDataFilesDataSequenceNumber; private final Map newDeleteFilesBySpec = Maps.newHashMap(); + private final Set newDVRefs = Sets.newHashSet(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); private final SnapshotSummary.Builder addedFilesSummary = SnapshotSummary.builder(); @@ -245,13 +251,13 @@ private PartitionSpec spec(int specId) { /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file) { - Preconditions.checkNotNull(file, "Invalid delete file: null"); + validateNewDeleteFile(file); add(new PendingDeleteFile(file)); } /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file, long dataSequenceNumber) { - Preconditions.checkNotNull(file, "Invalid delete file: null"); + validateNewDeleteFile(file); add(new PendingDeleteFile(file, dataSequenceNumber)); } @@ -268,9 +274,39 @@ private void add(PendingDeleteFile file) { if (deleteFiles.add(file)) { addedFilesSummary.addedFile(spec, file); hasNewDeleteFiles = true; + if (ContentFileUtil.isDV(file)) { + newDVRefs.add(file.referencedDataFile()); + } + } + } + + protected void validateNewDeleteFile(DeleteFile file) { + Preconditions.checkNotNull(file, "Invalid delete file: null"); + switch (formatVersion()) { + case 1: + throw new IllegalArgumentException("Deletes are supported in V2 and above"); + case 2: + Preconditions.checkArgument( + file.content() == FileContent.EQUALITY_DELETES || !ContentFileUtil.isDV(file), + "Must not use DVs for position deletes in V2: %s", + ContentFileUtil.dvDesc(file)); + break; + case 3: + Preconditions.checkArgument( + file.content() == FileContent.EQUALITY_DELETES || ContentFileUtil.isDV(file), + "Must use DVs for position deletes in V%s: %s", + formatVersion(), + file.location()); + break; + default: + throw new IllegalArgumentException("Unsupported format version: " + formatVersion()); } } + private int formatVersion() { + return ops.current().formatVersion(); + } + /** Add all files in a manifest to the new snapshot. */ protected void add(ManifestFile manifest) { Preconditions.checkArgument( @@ -769,6 +805,58 @@ protected void validateDataFilesExist( } } + // validates there are no concurrently added DVs for referenced data files + protected void validateAddedDVs( + TableMetadata base, + Long startingSnapshotId, + Expression conflictDetectionFilter, + Snapshot parent) { + // skip if there is no current table state or this operation doesn't add new DVs + if (parent == null || newDVRefs.isEmpty()) { + return; + } + + Pair, Set> history = + validationHistory( + base, + startingSnapshotId, + VALIDATE_ADDED_DVS_OPERATIONS, + ManifestContent.DELETES, + parent); + List newDeleteManifests = history.first(); + Set newSnapshotIds = history.second(); + + Tasks.foreach(newDeleteManifests) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(workerPool()) + .run(manifest -> validateAddedDVs(manifest, conflictDetectionFilter, newSnapshotIds)); + } + + private void validateAddedDVs( + ManifestFile manifest, Expression conflictDetectionFilter, Set newSnapshotIds) { + try (CloseableIterable> entries = + ManifestFiles.readDeleteManifest(manifest, ops.io(), ops.current().specsById()) + .filterRows(conflictDetectionFilter) + .caseSensitive(caseSensitive) + .liveEntries()) { + + for (ManifestEntry entry : entries) { + DeleteFile file = entry.file(); + if (newSnapshotIds.contains(entry.snapshotId()) && ContentFileUtil.isDV(file)) { + ValidationException.check( + !newDVRefs.contains(file.referencedDataFile()), + "Found concurrently added DV for %s: %s", + file.referencedDataFile(), + ContentFileUtil.dvDesc(file)); + } + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + // returns newly added manifests and snapshot IDs between the starting and parent snapshots private Pair, Set> validationHistory( TableMetadata base, Long startingSnapshotId, diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java index 6ef28191e78e..481422457b73 100644 --- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java @@ -53,15 +53,6 @@ public static List parameters() { return Arrays.asList(2, 3); } - static final DeleteFile FILE_A_POS_1 = - FileMetadata.deleteFileBuilder(SPEC) - .ofPositionDeletes() - .withPath("/path/to/data-a-pos-deletes.parquet") - .withFileSizeInBytes(10) - .withPartition(FILE_A.partition()) - .withRecordCount(1) - .build(); - static final DeleteFile FILE_A_EQ_1 = FileMetadata.deleteFileBuilder(SPEC) .ofEqualityDeletes() @@ -311,7 +302,7 @@ public void testUnpartitionedTableScan() throws IOException { public void testPartitionedTableWithPartitionPosDeletes() { table.newAppend().appendFile(FILE_A).commit(); - table.newRowDelta().addDeletes(FILE_A_POS_1).commit(); + table.newRowDelta().addDeletes(fileADeletes()).commit(); List tasks = Lists.newArrayList(newScan(table).planFiles().iterator()); assertThat(tasks).as("Should have one task").hasSize(1); @@ -323,7 +314,7 @@ public void testPartitionedTableWithPartitionPosDeletes() { assertThat(task.deletes()).as("Should have one associated delete file").hasSize(1); assertThat(task.deletes().get(0).path()) .as("Should have only pos delete file") - .isEqualTo(FILE_A_POS_1.path()); + .isEqualTo(fileADeletes().path()); } @TestTemplate @@ -349,7 +340,7 @@ public void testPartitionedTableWithPartitionEqDeletes() { public void testPartitionedTableWithUnrelatedPartitionDeletes() { table.newAppend().appendFile(FILE_B).commit(); - table.newRowDelta().addDeletes(FILE_A_POS_1).addDeletes(FILE_A_EQ_1).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_A_EQ_1).commit(); List tasks = Lists.newArrayList(newScan(table).planFiles().iterator()); assertThat(tasks).as("Should have one task").hasSize(1); @@ -363,7 +354,9 @@ public void testPartitionedTableWithUnrelatedPartitionDeletes() { @TestTemplate public void testPartitionedTableWithOlderPartitionDeletes() { - table.newRowDelta().addDeletes(FILE_A_POS_1).addDeletes(FILE_A_EQ_1).commit(); + assumeThat(formatVersion).as("DVs are not filtered using sequence numbers").isEqualTo(2); + + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A_EQ_1).commit(); table.newAppend().appendFile(FILE_A).commit(); @@ -379,6 +372,8 @@ public void testPartitionedTableWithOlderPartitionDeletes() { @TestTemplate public void testPartitionedTableScanWithGlobalDeletes() { + assumeThat(formatVersion).as("Requires V2 position deletes").isEqualTo(2); + table.newAppend().appendFile(FILE_A).commit(); TableMetadata base = table.ops().current(); @@ -407,6 +402,8 @@ public void testPartitionedTableScanWithGlobalDeletes() { @TestTemplate public void testPartitionedTableScanWithGlobalAndPartitionDeletes() { + assumeThat(formatVersion).as("Requires V2 position deletes").isEqualTo(2); + table.newAppend().appendFile(FILE_A).commit(); table.newRowDelta().addDeletes(FILE_A_EQ_1).commit(); @@ -437,7 +434,7 @@ public void testPartitionedTableScanWithGlobalAndPartitionDeletes() { @TestTemplate public void testPartitionedTableSequenceNumbers() { - table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_EQ_1).addDeletes(FILE_A_POS_1).commit(); + table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_EQ_1).addDeletes(fileADeletes()).commit(); List tasks = Lists.newArrayList(newScan(table).planFiles().iterator()); assertThat(tasks).as("Should have one task").hasSize(1); @@ -449,7 +446,7 @@ public void testPartitionedTableSequenceNumbers() { assertThat(task.deletes()).as("Should have one associated delete file").hasSize(1); assertThat(task.deletes().get(0).path()) .as("Should have only pos delete file") - .isEqualTo(FILE_A_POS_1.path()); + .isEqualTo(fileADeletes().path()); } @TestTemplate @@ -501,7 +498,7 @@ public void testPartitionedTableWithExistingDeleteFile() { table.newRowDelta().addDeletes(FILE_A_EQ_1).commit(); - table.newRowDelta().addDeletes(FILE_A_POS_1).commit(); + table.newRowDelta().addDeletes(fileADeletes()).commit(); table .updateProperties() @@ -557,7 +554,7 @@ public void testPartitionedTableWithExistingDeleteFile() { assertThat(task.deletes()).as("Should have two associated delete files").hasSize(2); assertThat(Sets.newHashSet(Iterables.transform(task.deletes(), ContentFile::path))) .as("Should have expected delete files") - .isEqualTo(Sets.newHashSet(FILE_A_EQ_1.path(), FILE_A_POS_1.path())); + .isEqualTo(Sets.newHashSet(FILE_A_EQ_1.path(), fileADeletes().path())); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java index 13e96869b454..80551f0a2247 100644 --- a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java @@ -186,7 +186,7 @@ public void scanningWithDeletes() throws IOException { reporter); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit(); - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(fileBDeletes()).commit(); ScanT tableScan = newScan(table); try (CloseableIterable fileScanTasks = tableScan.planFiles()) { @@ -208,12 +208,19 @@ public void scanningWithDeletes() throws IOException { assertThat(result.totalDataManifests().value()).isEqualTo(1); assertThat(result.totalDeleteManifests().value()).isEqualTo(1); assertThat(result.totalFileSizeInBytes().value()).isEqualTo(30L); - assertThat(result.totalDeleteFileSizeInBytes().value()).isEqualTo(20L); + assertThat(result.totalDeleteFileSizeInBytes().value()) + .isEqualTo(contentSize(fileADeletes(), fileBDeletes())); assertThat(result.skippedDataFiles().value()).isEqualTo(0); assertThat(result.skippedDeleteFiles().value()).isEqualTo(0); assertThat(result.indexedDeleteFiles().value()).isEqualTo(2); assertThat(result.equalityDeleteFiles().value()).isEqualTo(0); - assertThat(result.positionalDeleteFiles().value()).isEqualTo(2); + if (formatVersion == 2) { + assertThat(result.positionalDeleteFiles().value()).isEqualTo(2); + assertThat(result.dvs().value()).isEqualTo(0); + } else { + assertThat(result.positionalDeleteFiles().value()).isEqualTo(0); + assertThat(result.dvs().value()).isEqualTo(2); + } } @TestTemplate @@ -264,8 +271,8 @@ public void scanningWithSkippedDeleteFiles() throws IOException { tableDir, tableName, SCHEMA, SPEC, SortOrder.unsorted(), formatVersion, reporter); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_D).commit(); table.newOverwrite().deleteFile(FILE_A).addFile(FILE_A2).commit(); - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_D2_DELETES).commit(); - table.newRowDelta().addDeletes(FILE_B_DELETES).addDeletes(FILE_C2_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_D2_DELETES).commit(); + table.newRowDelta().addDeletes(fileBDeletes()).addDeletes(FILE_C2_DELETES).commit(); ScanT tableScan = newScan(table); List fileTasks = Lists.newArrayList(); @@ -308,7 +315,7 @@ public void scanningWithEqualityAndPositionalDeleteFiles() throws IOException { tableDir, tableName, SCHEMA, SPEC, SortOrder.unsorted(), formatVersion, reporter); table.newAppend().appendFile(FILE_A).commit(); // FILE_A_DELETES = positionalDelete / FILE_A2_DELETES = equalityDelete - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_A2_DELETES).commit(); ScanT tableScan = newScan(table); try (CloseableIterable fileScanTasks = @@ -321,7 +328,13 @@ public void scanningWithEqualityAndPositionalDeleteFiles() throws IOException { ScanMetricsResult result = scanReport.scanMetrics(); assertThat(result.indexedDeleteFiles().value()).isEqualTo(2); assertThat(result.equalityDeleteFiles().value()).isEqualTo(1); - assertThat(result.positionalDeleteFiles().value()).isEqualTo(1); + if (formatVersion == 2) { + assertThat(result.positionalDeleteFiles().value()).isEqualTo(1); + assertThat(result.dvs().value()).isEqualTo(0); + } else { + assertThat(result.positionalDeleteFiles().value()).isEqualTo(0); + assertThat(result.dvs().value()).isEqualTo(1); + } } static class TestMetricsReporter implements MetricsReporter { diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 9813d02910a6..46a1518e877f 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -45,6 +45,7 @@ import org.apache.iceberg.relocated.com.google.common.io.Files; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ScanTaskUtil; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.extension.ExtendWith; @@ -85,6 +86,17 @@ public class TestBase { .withPartitionPath("data_bucket=0") // easy way to set partition data for now .withRecordCount(1) .build(); + static final DeleteFile FILE_A_DV = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a-deletes.puffin") + .withFileSizeInBytes(10) + .withPartitionPath("data_bucket=0") + .withRecordCount(1) + .withReferencedDataFile(FILE_A.location()) + .withContentOffset(4) + .withContentSizeInBytes(6) + .build(); // Equality delete files. static final DeleteFile FILE_A2_DELETES = FileMetadata.deleteFileBuilder(SPEC) @@ -110,6 +122,17 @@ public class TestBase { .withPartitionPath("data_bucket=1") // easy way to set partition data for now .withRecordCount(1) .build(); + static final DeleteFile FILE_B_DV = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b-deletes.puffin") + .withFileSizeInBytes(10) + .withPartitionPath("data_bucket=1") + .withRecordCount(1) + .withReferencedDataFile(FILE_B.location()) + .withContentOffset(4) + .withContentSizeInBytes(6) + .build(); static final DataFile FILE_C = DataFiles.builder(SPEC) .withPath("/path/to/data-c.parquet") @@ -643,6 +666,22 @@ protected DataFile newDataFile(String partitionPath) { .build(); } + protected DeleteFile fileADeletes() { + return formatVersion >= 3 ? FILE_A_DV : FILE_A_DELETES; + } + + protected DeleteFile fileBDeletes() { + return formatVersion >= 3 ? FILE_B_DV : FILE_B_DELETES; + } + + protected DeleteFile newDeletes(DataFile dataFile) { + if (formatVersion >= 3) { + return FileGenerationUtil.generateDV(table, dataFile); + } else { + return FileGenerationUtil.generatePositionDeleteFile(table, dataFile); + } + } + protected DeleteFile newDeleteFile(int specId, String partitionPath) { PartitionSpec spec = table.specs().get(specId); return FileMetadata.deleteFileBuilder(spec) @@ -764,6 +803,14 @@ static Iterator files(ManifestFile manifest) { return ManifestFiles.read(manifest, FILE_IO).iterator(); } + static long recordCount(ContentFile... files) { + return Arrays.stream(files).mapToLong(ContentFile::recordCount).sum(); + } + + static long contentSize(ContentFile... files) { + return ScanTaskUtil.contentSizeInBytes(Arrays.asList(files)); + } + /** Used for assertions that only apply if the table version is v2. */ protected static class TableAssertions { private boolean enabled; diff --git a/core/src/test/java/org/apache/iceberg/TestBatchScans.java b/core/src/test/java/org/apache/iceberg/TestBatchScans.java index 1597f44f6338..72cd00e0573d 100644 --- a/core/src/test/java/org/apache/iceberg/TestBatchScans.java +++ b/core/src/test/java/org/apache/iceberg/TestBatchScans.java @@ -42,7 +42,7 @@ public void testDataTableScan() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); if (formatVersion > 1) { - table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).commit(); } BatchScan scan = table.newBatchScan(); diff --git a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java index 41b301668722..d333af98d623 100644 --- a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java +++ b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java @@ -95,11 +95,13 @@ public void addAndDeleteDeleteFiles() { // 2 positional + 1 equality table .newRowDelta() - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) + .addDeletes(fileADeletes()) + .addDeletes(fileBDeletes()) .addDeletes(FILE_C2_DELETES) .commit(); + long totalDeleteContentSize = contentSize(fileADeletes(), fileBDeletes(), FILE_C2_DELETES); + CommitReport report = reporter.lastCommitReport(); assertThat(report).isNotNull(); assertThat(report.operation()).isEqualTo("delete"); @@ -110,7 +112,13 @@ public void addAndDeleteDeleteFiles() { CommitMetricsResult metrics = report.commitMetrics(); assertThat(metrics.addedDeleteFiles().value()).isEqualTo(3L); assertThat(metrics.totalDeleteFiles().value()).isEqualTo(3L); - assertThat(metrics.addedPositionalDeleteFiles().value()).isEqualTo(2L); + if (formatVersion == 2) { + assertThat(metrics.addedPositionalDeleteFiles().value()).isEqualTo(2L); + assertThat(metrics.addedDVs()).isNull(); + } else { + assertThat(metrics.addedPositionalDeleteFiles()).isNull(); + assertThat(metrics.addedDVs().value()).isEqualTo(2L); + } assertThat(metrics.addedEqualityDeleteFiles().value()).isEqualTo(1L); assertThat(metrics.addedPositionalDeletes().value()).isEqualTo(2L); @@ -119,15 +127,15 @@ public void addAndDeleteDeleteFiles() { assertThat(metrics.addedEqualityDeletes().value()).isEqualTo(1L); assertThat(metrics.totalEqualityDeletes().value()).isEqualTo(1L); - assertThat(metrics.addedFilesSizeInBytes().value()).isEqualTo(30L); - assertThat(metrics.totalFilesSizeInBytes().value()).isEqualTo(30L); + assertThat(metrics.addedFilesSizeInBytes().value()).isEqualTo(totalDeleteContentSize); + assertThat(metrics.totalFilesSizeInBytes().value()).isEqualTo(totalDeleteContentSize); // now remove those 2 positional + 1 equality delete files table .newRewrite() .rewriteFiles( ImmutableSet.of(), - ImmutableSet.of(FILE_A_DELETES, FILE_B_DELETES, FILE_C2_DELETES), + ImmutableSet.of(fileADeletes(), fileBDeletes(), FILE_C2_DELETES), ImmutableSet.of(), ImmutableSet.of()) .commit(); @@ -142,7 +150,13 @@ public void addAndDeleteDeleteFiles() { metrics = report.commitMetrics(); assertThat(metrics.removedDeleteFiles().value()).isEqualTo(3L); assertThat(metrics.totalDeleteFiles().value()).isEqualTo(0L); - assertThat(metrics.removedPositionalDeleteFiles().value()).isEqualTo(2L); + if (formatVersion == 2) { + assertThat(metrics.removedPositionalDeleteFiles().value()).isEqualTo(2L); + assertThat(metrics.removedDVs()).isNull(); + } else { + assertThat(metrics.removedPositionalDeleteFiles()).isNull(); + assertThat(metrics.removedDVs().value()).isEqualTo(2L); + } assertThat(metrics.removedEqualityDeleteFiles().value()).isEqualTo(1L); assertThat(metrics.removedPositionalDeletes().value()).isEqualTo(2L); @@ -151,7 +165,7 @@ public void addAndDeleteDeleteFiles() { assertThat(metrics.removedEqualityDeletes().value()).isEqualTo(1L); assertThat(metrics.totalEqualityDeletes().value()).isEqualTo(0L); - assertThat(metrics.removedFilesSizeInBytes().value()).isEqualTo(30L); + assertThat(metrics.removedFilesSizeInBytes().value()).isEqualTo(totalDeleteContentSize); assertThat(metrics.totalFilesSizeInBytes().value()).isEqualTo(0L); } diff --git a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java index 9bce4e60a4f3..e061567e72a8 100644 --- a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java +++ b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java @@ -131,7 +131,7 @@ public void testEntriesTableWithDeleteManifests() { assumeThat(formatVersion).as("Only V2 Tables Support Deletes").isGreaterThanOrEqualTo(2); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).commit(); Table entriesTable = new ManifestEntriesTable(table); TableScan scan = entriesTable.newScan(); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 30fdae01cd94..f811dac02043 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -58,14 +58,14 @@ private void preparePartitionedTable(boolean transactional) { if (transactional) { table .newRowDelta() - .addDeletes(FILE_A_DELETES) - .addDeletes(FILE_B_DELETES) + .addDeletes(fileADeletes()) + .addDeletes(fileBDeletes()) .addDeletes(FILE_C2_DELETES) .addDeletes(FILE_D2_DELETES) .commit(); } else { - table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); - table.newRowDelta().addDeletes(FILE_B_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).commit(); + table.newRowDelta().addDeletes(fileBDeletes()).commit(); table.newRowDelta().addDeletes(FILE_C2_DELETES).commit(); table.newRowDelta().addDeletes(FILE_D2_DELETES).commit(); } @@ -721,7 +721,7 @@ public void testDeleteFilesTableSelection() throws IOException { assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); table.newFastAppend().appendFile(FILE_A).commit(); - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_A2_DELETES).commit(); Table deleteFilesTable = new DeleteFilesTable(table); @@ -1409,10 +1409,10 @@ public void testPositionDeletesWithFilter() { .containsEntry(MetadataColumns.SPEC_ID.fieldId(), 0); assertThat(posDeleteTask.file().path()) .as("Expected correct delete file on task") - .isEqualTo(FILE_B_DELETES.path()); + .isEqualTo(fileBDeletes().path()); assertThat((Map) constantsMap(posDeleteTask, partitionType)) .as("Expected correct delete file on constant column") - .containsEntry(MetadataColumns.FILE_PATH.fieldId(), FILE_B_DELETES.path().toString()); + .containsEntry(MetadataColumns.FILE_PATH.fieldId(), fileBDeletes().path().toString()); } @TestTemplate @@ -1479,17 +1479,16 @@ private void testPositionDeletesBaseTableFilter(boolean transactional) { .containsEntry(MetadataColumns.SPEC_ID.fieldId(), 0); assertThat(posDeleteTask.file().path()) .as("Expected correct delete file on task") - .isEqualTo(FILE_A_DELETES.path()); + .isEqualTo(fileADeletes().path()); assertThat((Map) constantsMap(posDeleteTask, partitionType)) .as("Expected correct delete file on constant column") - .containsEntry(MetadataColumns.FILE_PATH.fieldId(), FILE_A_DELETES.path().toString()); + .containsEntry(MetadataColumns.FILE_PATH.fieldId(), fileADeletes().path().toString()); } @TestTemplate public void testPositionDeletesWithBaseTableFilterNot() { - assumeThat(formatVersion) - .as("Position deletes are not supported by V1 Tables") - .isNotEqualTo(1); // use identity rather than bucket partition spec, + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isEqualTo(2); + // use identity rather than bucket partition spec, // as bucket.project does not support projecting notEq table.updateSpec().removeField("data_bucket").addField("id").commit(); PartitionSpec spec = table.spec(); @@ -1619,20 +1618,8 @@ public void testPositionDeletesUnpartitioned() { .build(); table.newAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); - DeleteFile delete1 = - FileMetadata.deleteFileBuilder(table.spec()) - .ofPositionDeletes() - .withPath("/path/to/delete1.parquet") - .withFileSizeInBytes(10) - .withRecordCount(1) - .build(); - DeleteFile delete2 = - FileMetadata.deleteFileBuilder(table.spec()) - .ofPositionDeletes() - .withPath("/path/to/delete2.parquet") - .withFileSizeInBytes(10) - .withRecordCount(1) - .build(); + DeleteFile delete1 = newDeletes(dataFile1); + DeleteFile delete2 = newDeletes(dataFile2); table.newRowDelta().addDeletes(delete1).addDeletes(delete2).commit(); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1655,16 +1642,16 @@ public void testPositionDeletesUnpartitioned() { .isEqualTo(1); assertThat(scanTasks).hasSize(2); - scanTasks.sort(Comparator.comparing(f -> f.file().path().toString())); - assertThat(scanTasks.get(0).file().path().toString()).isEqualTo("/path/to/delete1.parquet"); - assertThat(scanTasks.get(1).file().path().toString()).isEqualTo("/path/to/delete2.parquet"); + scanTasks.sort(Comparator.comparing(f -> f.file().pos())); + assertThat(scanTasks.get(0).file().location()).isEqualTo(delete1.location()); + assertThat(scanTasks.get(1).file().location()).isEqualTo(delete2.location()); Types.StructType partitionType = Partitioning.partitionType(table); assertThat((Map) constantsMap(scanTasks.get(0), partitionType)) - .containsEntry(MetadataColumns.FILE_PATH.fieldId(), "/path/to/delete1.parquet"); + .containsEntry(MetadataColumns.FILE_PATH.fieldId(), delete1.location()); assertThat((Map) constantsMap(scanTasks.get(1), partitionType)) - .containsEntry(MetadataColumns.FILE_PATH.fieldId(), "/path/to/delete2.parquet"); + .containsEntry(MetadataColumns.FILE_PATH.fieldId(), delete2.location()); assertThat((Map) constantsMap(scanTasks.get(0), partitionType)) .containsEntry(MetadataColumns.SPEC_ID.fieldId(), 1); assertThat((Map) constantsMap(scanTasks.get(1), partitionType)) @@ -1712,20 +1699,8 @@ public void testPositionDeletesManyColumns() { .build(); table.newAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); - DeleteFile delete1 = - FileMetadata.deleteFileBuilder(table.spec()) - .ofPositionDeletes() - .withPath("/path/to/delete1.parquet") - .withFileSizeInBytes(10) - .withRecordCount(1) - .build(); - DeleteFile delete2 = - FileMetadata.deleteFileBuilder(table.spec()) - .ofPositionDeletes() - .withPath("/path/to/delete2.parquet") - .withFileSizeInBytes(10) - .withRecordCount(1) - .build(); + DeleteFile delete1 = newDeletes(dataFile1); + DeleteFile delete2 = newDeletes(dataFile2); table.newRowDelta().addDeletes(delete1).addDeletes(delete2).commit(); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1745,9 +1720,9 @@ public void testPositionDeletesManyColumns() { return (PositionDeletesScanTask) task; })); assertThat(scanTasks).hasSize(2); - scanTasks.sort(Comparator.comparing(f -> f.file().path().toString())); - assertThat(scanTasks.get(0).file().path().toString()).isEqualTo("/path/to/delete1.parquet"); - assertThat(scanTasks.get(1).file().path().toString()).isEqualTo("/path/to/delete2.parquet"); + scanTasks.sort(Comparator.comparing(f -> f.file().pos())); + assertThat(scanTasks.get(0).file().location()).isEqualTo(delete1.location()); + assertThat(scanTasks.get(1).file().location()).isEqualTo(delete2.location()); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java index 2de38541777b..84860d34bb31 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java @@ -159,7 +159,7 @@ public void testPartitionsTableScanWithAddPartitionOnNestedField() { @TestTemplate public void testPositionDeletesPartitionSpecRemoval() { - assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isEqualTo(2); table.updateSpec().removeField("id").commit(); DeleteFile deleteFile = newDeleteFile(table.ops().current().spec().specId(), "nested.id=1"); diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java index 124cc2f28dd5..5b108e9ee565 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java @@ -55,6 +55,7 @@ protected static List parameters() { @TestTemplate public void testEmptyTable() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); @@ -87,6 +88,7 @@ public void testEmptyTable() { @TestTemplate public void testAddOnly() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); assertThat(listManifestFiles()).isEmpty(); assertThatThrownBy( @@ -130,6 +132,7 @@ public void testAddOnly() { @TestTemplate public void testDeleteOnly() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); assertThat(listManifestFiles()).isEmpty(); assertThatThrownBy( diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index f1d23de32a42..72bb85c0446e 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -1096,7 +1096,7 @@ public void testRewriteDataManifestsPreservesDeletes() { long appendSnapshotSeq = appendSnapshot.sequenceNumber(); // commit delete files - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_A2_DELETES).commit(); // save the delete snapshot info Snapshot deleteSnapshot = table.currentSnapshot(); @@ -1139,7 +1139,7 @@ public void testRewriteDataManifestsPreservesDeletes() { dataSeqs(deleteSnapshotSeq, deleteSnapshotSeq), fileSeqs(deleteSnapshotSeq, deleteSnapshotSeq), ids(deleteSnapshotId, deleteSnapshotId), - files(FILE_A_DELETES, FILE_A2_DELETES), + files(fileADeletes(), FILE_A2_DELETES), statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED)); } @@ -1158,7 +1158,7 @@ public void testReplaceDeleteManifestsOnly() throws IOException { long appendSnapshotSeq = appendSnapshot.sequenceNumber(); // commit delete files - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_A2_DELETES).commit(); // save the delete snapshot info Snapshot deleteSnapshot = table.currentSnapshot(); @@ -1179,7 +1179,7 @@ public void testReplaceDeleteManifestsOnly() throws IOException { deleteSnapshotId, deleteSnapshotSeq, deleteSnapshotSeq, - FILE_A_DELETES)); + fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( "delete-manifest-file-2.avro", @@ -1218,7 +1218,7 @@ public void testReplaceDeleteManifestsOnly() throws IOException { dataSeqs(deleteSnapshotSeq), fileSeqs(deleteSnapshotSeq), ids(deleteSnapshotId), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(ManifestEntry.Status.EXISTING)); validateDeleteManifest( deleteManifests.get(1), @@ -1244,7 +1244,7 @@ public void testReplaceDataAndDeleteManifests() throws IOException { long appendSnapshotSeq = appendSnapshot.sequenceNumber(); // commit delete files - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_A2_DELETES).commit(); // save the delete snapshot info Snapshot deleteSnapshot = table.currentSnapshot(); @@ -1287,7 +1287,7 @@ public void testReplaceDataAndDeleteManifests() throws IOException { deleteSnapshotId, deleteSnapshotSeq, deleteSnapshotSeq, - FILE_A_DELETES)); + fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( "delete-manifest-file-2.avro", @@ -1337,7 +1337,7 @@ public void testReplaceDataAndDeleteManifests() throws IOException { dataSeqs(deleteSnapshotSeq), fileSeqs(deleteSnapshotSeq), ids(deleteSnapshotId), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(ManifestEntry.Status.EXISTING)); validateDeleteManifest( deleteManifests.get(1), @@ -1361,7 +1361,7 @@ public void testDeleteManifestReplacementConcurrentAppend() throws IOException { long appendSnapshotSeq = appendSnapshot.sequenceNumber(); // commit delete files - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_A2_DELETES).commit(); // save the delete snapshot info Snapshot deleteSnapshot = table.currentSnapshot(); @@ -1379,7 +1379,7 @@ public void testDeleteManifestReplacementConcurrentAppend() throws IOException { deleteSnapshotId, deleteSnapshotSeq, deleteSnapshotSeq, - FILE_A_DELETES)); + fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( "delete-manifest-file-2.avro", @@ -1440,7 +1440,7 @@ public void testDeleteManifestReplacementConcurrentAppend() throws IOException { dataSeqs(deleteSnapshotSeq), fileSeqs(deleteSnapshotSeq), ids(deleteSnapshotId), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(ManifestEntry.Status.EXISTING)); validateDeleteManifest( deleteManifests.get(1), @@ -1464,7 +1464,7 @@ public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IO long appendSnapshotSeq = appendSnapshot.sequenceNumber(); // commit the first set of delete files - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_A2_DELETES).commit(); // save the first delete snapshot info Snapshot deleteSnapshot1 = table.currentSnapshot(); @@ -1472,7 +1472,7 @@ public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IO long deleteSnapshotSeq1 = deleteSnapshot1.sequenceNumber(); // commit the second set of delete files - table.newRowDelta().addDeletes(FILE_B_DELETES).addDeletes(FILE_C2_DELETES).commit(); + table.newRowDelta().addDeletes(fileBDeletes()).addDeletes(FILE_C2_DELETES).commit(); // save the second delete snapshot info Snapshot deleteSnapshot2 = table.currentSnapshot(); @@ -1489,7 +1489,7 @@ public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IO deleteSnapshotId1, deleteSnapshotSeq1, deleteSnapshotSeq1, - FILE_A_DELETES)); + fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( "delete-manifest-file-2.avro", @@ -1507,7 +1507,7 @@ public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IO rewriteManifests.addManifest(newDeleteManifest2); // commit the third set of delete files concurrently - table.newRewrite().deleteFile(FILE_B_DELETES).commit(); + table.newRewrite().deleteFile(fileBDeletes()).commit(); Snapshot concurrentSnapshot = table.currentSnapshot(); long concurrentSnapshotId = concurrentSnapshot.snapshotId(); @@ -1541,7 +1541,7 @@ public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IO dataSeqs(deleteSnapshotSeq1), fileSeqs(deleteSnapshotSeq1), ids(deleteSnapshotId1), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(ManifestEntry.Status.EXISTING)); validateDeleteManifest( deleteManifests.get(1), @@ -1555,7 +1555,7 @@ public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IO dataSeqs(deleteSnapshotSeq2, deleteSnapshotSeq2), fileSeqs(deleteSnapshotSeq2, deleteSnapshotSeq2), ids(concurrentSnapshotId, deleteSnapshotId2), - files(FILE_B_DELETES, FILE_C2_DELETES), + files(fileBDeletes(), FILE_C2_DELETES), statuses(ManifestEntry.Status.DELETED, ManifestEntry.Status.EXISTING)); } @@ -1567,7 +1567,7 @@ public void testDeleteManifestReplacementConflictingDeleteFileRemoval() throws I table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit(); // commit delete files - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(FILE_A2_DELETES).commit(); // save the delete snapshot info Snapshot deleteSnapshot = table.currentSnapshot(); @@ -1584,7 +1584,7 @@ public void testDeleteManifestReplacementConflictingDeleteFileRemoval() throws I deleteSnapshotId, deleteSnapshotSeq, deleteSnapshotSeq, - FILE_A_DELETES)); + fileADeletes())); ManifestFile newDeleteManifest2 = writeManifest( "delete-manifest-file-2.avro", @@ -1602,7 +1602,7 @@ public void testDeleteManifestReplacementConflictingDeleteFileRemoval() throws I rewriteManifests.addManifest(newDeleteManifest2); // modify the original delete manifest concurrently - table.newRewrite().deleteFile(FILE_A_DELETES).commit(); + table.newRewrite().deleteFile(fileADeletes()).commit(); // the rewrite must fail as the original delete manifest was replaced concurrently assertThatThrownBy(rewriteManifests::commit) @@ -1621,7 +1621,7 @@ public void testDeleteManifestReplacementFailure() throws IOException { table.newFastAppend().appendFile(FILE_A).commit(); // commit the first delete file - table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).commit(); // save the first delete snapshot info Snapshot deleteSnapshot1 = table.currentSnapshot(); @@ -1648,7 +1648,7 @@ public void testDeleteManifestReplacementFailure() throws IOException { deleteSnapshotId1, deleteSnapshotSeq1, deleteSnapshotSeq1, - FILE_A_DELETES), + fileADeletes()), manifestEntry( ManifestEntry.Status.EXISTING, deleteSnapshotId2, diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 1d67e48a2ce2..b41be0c7a636 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -29,7 +29,9 @@ import static org.apache.iceberg.util.SnapshotUtil.latestSnapshot; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; +import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -38,8 +40,11 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -52,13 +57,17 @@ public class TestRowDelta extends V2TableTestBase { @Parameters(name = "formatVersion = {0}, branch = {1}") protected static List parameters() { - return Arrays.asList(new Object[] {2, "main"}, new Object[] {2, "testBranch"}); + return Arrays.asList( + new Object[] {2, "main"}, + new Object[] {2, "testBranch"}, + new Object[] {3, "main"}, + new Object[] {3, "testBranch"}); } @TestTemplate public void addOnlyDeleteFilesProducesDeleteOperation() { SnapshotUpdate rowDelta = - table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES); + table.newRowDelta().addDeletes(fileADeletes()).addDeletes(fileBDeletes()); commit(table, rowDelta, branch); Snapshot snap = latestSnapshot(table, branch); @@ -70,7 +79,7 @@ public void addOnlyDeleteFilesProducesDeleteOperation() { @TestTemplate public void testAddDeleteFile() { SnapshotUpdate rowDelta = - table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES); + table.newRowDelta().addRows(FILE_A).addDeletes(fileADeletes()).addDeletes(fileBDeletes()); commit(table, rowDelta, branch); Snapshot snap = latestSnapshot(table, branch); @@ -95,7 +104,7 @@ public void testAddDeleteFile() { dataSeqs(1L, 1L), fileSeqs(1L, 1L), ids(snap.snapshotId(), snap.snapshotId()), - files(FILE_A_DELETES, FILE_B_DELETES), + files(fileADeletes(), fileBDeletes()), statuses(Status.ADDED, Status.ADDED)); } @@ -126,7 +135,7 @@ public void testValidateDataFilesExistDefaults() { table, table .newRowDelta() - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) .validateDataFilesExist(ImmutableList.of(FILE_A.path())), branch)) @@ -143,7 +152,7 @@ public void testValidateDataFilesExistDefaults() { table, table .newRowDelta() - .addDeletes(FILE_B_DELETES) + .addDeletes(fileBDeletes()) .validateDataFilesExist(ImmutableList.of(FILE_B.path())) .validateFromSnapshot(validateFromSnapshotId), branch); @@ -155,7 +164,7 @@ public void testValidateDataFilesExistDefaults() { dataSeqs(4L), fileSeqs(4L), ids(latestSnapshot(table, branch).snapshotId()), - files(FILE_B_DELETES), + files(fileBDeletes()), statuses(Status.ADDED)); } @@ -177,7 +186,7 @@ public void testValidateDataFilesExistOverwrite() { table, table .newRowDelta() - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) .validateDataFilesExist(ImmutableList.of(FILE_A.path())), branch)) @@ -209,7 +218,7 @@ public void testValidateDataFilesExistReplacePartitions() { table, table .newRowDelta() - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) .validateDataFilesExist(ImmutableList.of(FILE_A.path())), branch)) @@ -242,7 +251,7 @@ public void testValidateDataFilesExistFromSnapshot() { table, table .newRowDelta() - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) .validateDataFilesExist(ImmutableList.of(FILE_A.path())), branch); @@ -276,7 +285,7 @@ public void testValidateDataFilesExistFromSnapshot() { dataSeqs(3L), fileSeqs(3L), ids(snap.snapshotId()), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(Status.ADDED)); } @@ -301,7 +310,7 @@ public void testValidateDataFilesExistRewrite() { table, table .newRowDelta() - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) .validateDataFilesExist(ImmutableList.of(FILE_A.path())), branch)) @@ -333,7 +342,7 @@ public void testValidateDataFilesExistValidateDeletes() { table, table .newRowDelta() - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateDeletedFiles() .validateFromSnapshot(validateFromSnapshotId) .validateDataFilesExist(ImmutableList.of(FILE_A.path())), @@ -366,7 +375,7 @@ public void testValidateNoConflicts() { table, table .newRowDelta() - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) .conflictDetectionFilter( Expressions.equal("data", "u")) // bucket16("u") -> 0 @@ -399,7 +408,7 @@ public void testValidateNoConflictsFromSnapshot() { table, table .newRowDelta() - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateDeletedFiles() .validateFromSnapshot(validateFromSnapshotId) .validateDataFilesExist(ImmutableList.of(FILE_A.path())) @@ -436,7 +445,7 @@ public void testValidateNoConflictsFromSnapshot() { dataSeqs(3L), fileSeqs(3L), ids(snap.snapshotId()), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(Status.ADDED)); } @@ -444,7 +453,7 @@ public void testValidateNoConflictsFromSnapshot() { public void testOverwriteWithDeleteFile() { commit( table, - table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES), + table.newRowDelta().addRows(FILE_A).addDeletes(fileADeletes()).addDeletes(fileBDeletes()), branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); @@ -479,7 +488,7 @@ public void testOverwriteWithDeleteFile() { dataSeqs(1L, 1L), fileSeqs(1L, 1L), ids(snap.snapshotId(), deltaSnapshotId), - files(FILE_A_DELETES, FILE_B_DELETES), + files(fileADeletes(), fileBDeletes()), statuses(Status.DELETED, Status.EXISTING)); } @@ -487,7 +496,7 @@ public void testOverwriteWithDeleteFile() { public void testReplacePartitionsWithDeleteFile() { commit( table, - table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES), + table.newRowDelta().addRows(FILE_A).addDeletes(fileADeletes()).addDeletes(fileBDeletes()), branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); @@ -526,7 +535,7 @@ public void testReplacePartitionsWithDeleteFile() { dataSeqs(1L, 1L), fileSeqs(1L, 1L), ids(snap.snapshotId(), deltaSnapshotId), - files(FILE_A_DELETES, FILE_B_DELETES), + files(fileADeletes(), fileBDeletes()), statuses(Status.DELETED, Status.EXISTING)); } @@ -534,7 +543,7 @@ public void testReplacePartitionsWithDeleteFile() { public void testDeleteByExpressionWithDeleteFile() { commit( table, - table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES), + table.newRowDelta().addRows(FILE_A).addDeletes(fileADeletes()).addDeletes(fileBDeletes()), branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); @@ -564,13 +573,13 @@ public void testDeleteByExpressionWithDeleteFile() { dataSeqs(1L, 1L), fileSeqs(1L, 1L), ids(snap.snapshotId(), snap.snapshotId()), - files(FILE_A_DELETES, FILE_B_DELETES), + files(fileADeletes(), fileBDeletes()), statuses(Status.DELETED, Status.DELETED)); } @TestTemplate public void testDeleteDataFileWithDeleteFile() { - commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); + commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(fileADeletes()), branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); assertThat(latestSnapshot(table, branch).sequenceNumber()).isEqualTo(1); @@ -598,7 +607,7 @@ public void testDeleteDataFileWithDeleteFile() { dataSeqs(1L), fileSeqs(1L), ids(deltaSnapshotId), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(Status.ADDED)); // the manifest that removed FILE_A will be dropped next commit, causing the min sequence number @@ -619,13 +628,13 @@ public void testDeleteDataFileWithDeleteFile() { dataSeqs(1L), fileSeqs(1L), ids(nextSnap.snapshotId()), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(Status.DELETED)); } @TestTemplate public void testFastAppendDoesNotRemoveStaleDeleteFiles() { - commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); + commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(fileADeletes()), branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); assertThat(latestSnapshot(table, branch).sequenceNumber()).isEqualTo(1); @@ -653,7 +662,7 @@ public void testFastAppendDoesNotRemoveStaleDeleteFiles() { dataSeqs(1L), fileSeqs(1L), ids(deltaSnapshotId), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(Status.ADDED)); // the manifest that removed FILE_A will be dropped next merging commit, but FastAppend will not @@ -689,7 +698,7 @@ public void testFastAppendDoesNotRemoveStaleDeleteFiles() { dataSeqs(1L), fileSeqs(1L), ids(deltaSnapshotId), - files(FILE_A_DELETES), + files(fileADeletes()), statuses(Status.ADDED)); } @@ -728,14 +737,7 @@ public void testValidateDataFilesExistWithConflictDetectionFilter() { Snapshot baseSnapshot = latestSnapshot(table, branch); // add a delete file for partition A - DeleteFile deleteFile = - FileMetadata.deleteFileBuilder(table.spec()) - .ofPositionDeletes() - .withPath("/path/to/data-a-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data=a") - .withRecordCount(1) - .build(); + DeleteFile deleteFile = newDeletes(dataFile1); Expression conflictDetectionFilter = Expressions.equal("data", "a"); RowDelta rowDelta = @@ -789,14 +791,7 @@ public void testValidateDataFilesDoNotExistWithConflictDetectionFilter() { Snapshot baseSnapshot = latestSnapshot(table, branch); // add a delete file for partition A - DeleteFile deleteFile = - FileMetadata.deleteFileBuilder(table.spec()) - .ofPositionDeletes() - .withPath("/path/to/data-a-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data=a") - .withRecordCount(1) - .build(); + DeleteFile deleteFile = newDeletes(dataFile1); Expression conflictDetectionFilter = Expressions.equal("data", "a"); RowDelta rowDelta = @@ -847,9 +842,9 @@ public void testAddDeleteFilesMultipleSpecs() { // commit a row delta with 1 data file and 3 delete files where delete files have different // specs DataFile dataFile = newDataFile("data=xyz"); - DeleteFile firstDeleteFile = newDeleteFile(firstSnapshotDataFile.specId(), "data_bucket=0"); - DeleteFile secondDeleteFile = newDeleteFile(secondSnapshotDataFile.specId(), ""); - DeleteFile thirdDeleteFile = newDeleteFile(thirdSnapshotDataFile.specId(), "data=abc"); + DeleteFile firstDeleteFile = newDeletes(firstSnapshotDataFile); + DeleteFile secondDeleteFile = newDeletes(secondSnapshotDataFile); + DeleteFile thirdDeleteFile = newDeletes(thirdSnapshotDataFile); commit( table, @@ -867,6 +862,7 @@ public void testAddDeleteFilesMultipleSpecs() { assertThat(snapshot.operation()).isEqualTo(DataOperations.OVERWRITE); Map summary = snapshot.summary(); + long posDeletesCount = recordCount(firstDeleteFile, secondDeleteFile, thirdDeleteFile); assertThat(summary) .containsEntry(CHANGED_PARTITION_COUNT_PROP, "4") @@ -874,8 +870,8 @@ public void testAddDeleteFilesMultipleSpecs() { .containsEntry(TOTAL_DATA_FILES_PROP, "4") .containsEntry(ADDED_DELETE_FILES_PROP, "3") .containsEntry(TOTAL_DELETE_FILES_PROP, "3") - .containsEntry(ADDED_POS_DELETES_PROP, "3") - .containsEntry(TOTAL_POS_DELETES_PROP, "3") + .containsEntry(ADDED_POS_DELETES_PROP, String.valueOf(posDeletesCount)) + .containsEntry(TOTAL_POS_DELETES_PROP, String.valueOf(posDeletesCount)) .hasEntrySatisfying( CHANGED_PARTITION_PREFIX + "data_bucket=0", v -> assertThat(v).contains(ADDED_DELETE_FILES_PROP + "=1")) @@ -953,8 +949,8 @@ public void testManifestMergingMultipleSpecs() { commit(table, table.newAppend().appendFile(secondSnapshotDataFile), branch); // commit two delete files to two specs in a single operation - DeleteFile firstDeleteFile = newDeleteFile(firstSnapshotDataFile.specId(), "data_bucket=0"); - DeleteFile secondDeleteFile = newDeleteFile(secondSnapshotDataFile.specId(), ""); + DeleteFile firstDeleteFile = newDeletes(firstSnapshotDataFile); + DeleteFile secondDeleteFile = newDeletes(secondSnapshotDataFile); commit( table, @@ -968,12 +964,18 @@ public void testManifestMergingMultipleSpecs() { assertThat(thirdSnapshot.deleteManifests(table.io())).hasSize(2); // commit two more delete files to the same specs to trigger merging - DeleteFile thirdDeleteFile = newDeleteFile(firstSnapshotDataFile.specId(), "data_bucket=0"); - DeleteFile fourthDeleteFile = newDeleteFile(secondSnapshotDataFile.specId(), ""); + DeleteFile thirdDeleteFile = newDeletes(firstSnapshotDataFile); + DeleteFile fourthDeleteFile = newDeletes(secondSnapshotDataFile); commit( table, - table.newRowDelta().addDeletes(thirdDeleteFile).addDeletes(fourthDeleteFile), + table + .newRowDelta() + .removeDeletes(firstDeleteFile) + .addDeletes(thirdDeleteFile) + .removeDeletes(secondDeleteFile) + .addDeletes(fourthDeleteFile) + .validateFromSnapshot(thirdSnapshot.snapshotId()), branch); Snapshot fourthSnapshot = latestSnapshot(table, branch); @@ -988,9 +990,9 @@ public void testManifestMergingMultipleSpecs() { firstDeleteManifest, dataSeqs(4L, 3L), fileSeqs(4L, 3L), - ids(fourthSnapshot.snapshotId(), thirdSnapshot.snapshotId()), + ids(fourthSnapshot.snapshotId(), fourthSnapshot.snapshotId()), files(thirdDeleteFile, firstDeleteFile), - statuses(Status.ADDED, Status.EXISTING)); + statuses(Status.ADDED, Status.DELETED)); ManifestFile secondDeleteManifest = fourthSnapshot.deleteManifests(table.io()).get(0); assertThat(secondDeleteManifest.partitionSpecId()).isEqualTo(secondSnapshotDataFile.specId()); @@ -998,9 +1000,9 @@ public void testManifestMergingMultipleSpecs() { secondDeleteManifest, dataSeqs(4L, 3L), fileSeqs(4L, 3L), - ids(fourthSnapshot.snapshotId(), thirdSnapshot.snapshotId()), + ids(fourthSnapshot.snapshotId(), fourthSnapshot.snapshotId()), files(fourthDeleteFile, secondDeleteFile), - statuses(Status.ADDED, Status.EXISTING)); + statuses(Status.ADDED, Status.DELETED)); } @TestTemplate @@ -1019,8 +1021,8 @@ public void testAbortMultipleSpecs() { commit(table, table.newAppend().appendFile(secondSnapshotDataFile), branch); // prepare two delete files that belong to different specs - DeleteFile firstDeleteFile = newDeleteFile(firstSnapshotDataFile.specId(), "data_bucket=0"); - DeleteFile secondDeleteFile = newDeleteFile(secondSnapshotDataFile.specId(), ""); + DeleteFile firstDeleteFile = newDeletes(firstSnapshotDataFile); + DeleteFile secondDeleteFile = newDeletes(secondSnapshotDataFile); // capture all deletes Set deletedFiles = Sets.newHashSet(); @@ -1062,7 +1064,7 @@ public void testConcurrentConflictingRowDelta() { .newRowDelta() .toBranch(branch) .addRows(FILE_B) - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(firstSnapshot.snapshotId()) .conflictDetectionFilter(conflictDetectionFilter) .validateNoConflictingDataFiles() @@ -1071,7 +1073,7 @@ public void testConcurrentConflictingRowDelta() { table .newRowDelta() .toBranch(branch) - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(firstSnapshot.snapshotId()) .conflictDetectionFilter(conflictDetectionFilter) .validateNoConflictingDataFiles() @@ -1094,7 +1096,7 @@ public void testConcurrentConflictingRowDeltaWithoutAppendValidation() { RowDelta rowDelta = table .newRowDelta() - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(firstSnapshot.snapshotId()) .conflictDetectionFilter(conflictDetectionFilter) .validateNoConflictingDeleteFiles(); @@ -1102,7 +1104,7 @@ public void testConcurrentConflictingRowDeltaWithoutAppendValidation() { table .newRowDelta() .toBranch(branch) - .addDeletes(FILE_A_DELETES) + .addDeletes(fileADeletes()) .validateFromSnapshot(firstSnapshot.snapshotId()) .conflictDetectionFilter(conflictDetectionFilter) .validateNoConflictingDataFiles() @@ -1149,14 +1151,7 @@ public void testConcurrentNonConflictingRowDelta() { Expression conflictDetectionFilter = Expressions.equal("data", "a"); // add a delete file for partition A - DeleteFile deleteFile1 = - FileMetadata.deleteFileBuilder(table.spec()) - .ofPositionDeletes() - .withPath("/path/to/data-a-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data=a") - .withRecordCount(1) - .build(); + DeleteFile deleteFile1 = newDeletes(dataFile1); // mock a DELETE operation with serializable isolation RowDelta rowDelta = @@ -1170,14 +1165,7 @@ public void testConcurrentNonConflictingRowDelta() { .validateNoConflictingDeleteFiles(); // add a delete file for partition B - DeleteFile deleteFile2 = - FileMetadata.deleteFileBuilder(table.spec()) - .ofPositionDeletes() - .withPath("/path/to/data-b-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data=b") - .withRecordCount(1) - .build(); + DeleteFile deleteFile2 = newDeletes(dataFile2); table .newRowDelta() @@ -1320,8 +1308,8 @@ public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() Snapshot baseSnapshot = latestSnapshot(table, branch); - // add an position delete file - DeleteFile deleteFile1 = newDeleteFile(table.spec().specId(), "data=a"); + // add position deletes + DeleteFile deleteFile1 = newDeletes(dataFile1); // mock a DELETE operation with serializable isolation RowDelta rowDelta = @@ -1357,7 +1345,7 @@ public void testRowDeltaCaseSensitivity() { Snapshot firstSnapshot = latestSnapshot(table, branch); - commit(table, table.newRowDelta().addDeletes(FILE_A_DELETES), branch); + commit(table, table.newRowDelta().addDeletes(fileADeletes()), branch); Expression conflictDetectionFilter = Expressions.equal(Expressions.bucket("dAtA", 16), 0); @@ -1413,12 +1401,12 @@ public void testRowDeltaCaseSensitivity() { @TestTemplate public void testRewrittenDeleteFiles() { DataFile dataFile = newDataFile("data_bucket=0"); - DeleteFile deleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + DeleteFile deleteFile = newDeletes(dataFile); RowDelta baseRowDelta = table.newRowDelta().addRows(dataFile).addDeletes(deleteFile); Snapshot baseSnapshot = commit(table, baseRowDelta, branch); assertThat(baseSnapshot.operation()).isEqualTo(DataOperations.OVERWRITE); - DeleteFile newDeleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + DeleteFile newDeleteFile = newDeletes(dataFile); RowDelta rowDelta = table .newRowDelta() @@ -1458,14 +1446,16 @@ public void testRewrittenDeleteFiles() { @TestTemplate public void testConcurrentDeletesRewriteSameDeleteFile() { + assumeThat(formatVersion).isEqualTo(2); + DataFile dataFile = newDataFile("data_bucket=0"); - DeleteFile deleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + DeleteFile deleteFile = newDeletes(dataFile); RowDelta baseRowDelta = table.newRowDelta().addRows(dataFile).addDeletes(deleteFile); Snapshot baseSnapshot = commit(table, baseRowDelta, branch); assertThat(baseSnapshot.operation()).isEqualTo(DataOperations.OVERWRITE); // commit the first DELETE operation that replaces `deleteFile` - DeleteFile newDeleteFile1 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + DeleteFile newDeleteFile1 = newDeletes(dataFile); RowDelta delete1 = table .newRowDelta() @@ -1478,7 +1468,7 @@ public void testConcurrentDeletesRewriteSameDeleteFile() { assertThat(snapshot1.sequenceNumber()).isEqualTo(2L); // commit the second DELETE operation that replaces `deleteFile` - DeleteFile newDeleteFile2 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + DeleteFile newDeleteFile2 = newDeletes(dataFile); RowDelta delete2 = table .newRowDelta() @@ -1522,13 +1512,13 @@ public void testConcurrentDeletesRewriteSameDeleteFile() { @TestTemplate public void testConcurrentMergeRewriteSameDeleteFile() { DataFile dataFile = newDataFile("data_bucket=0"); - DeleteFile deleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + DeleteFile deleteFile = newDeletes(dataFile); RowDelta baseRowDelta = table.newRowDelta().addRows(dataFile).addDeletes(deleteFile); Snapshot baseSnapshot = commit(table, baseRowDelta, branch); assertThat(baseSnapshot.operation()).isEqualTo(DataOperations.OVERWRITE); // commit a DELETE operation that replaces `deleteFile` - DeleteFile newDeleteFile1 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + DeleteFile newDeleteFile1 = newDeletes(dataFile); RowDelta delete = table .newRowDelta() @@ -1540,7 +1530,7 @@ public void testConcurrentMergeRewriteSameDeleteFile() { // attempt to commit a MERGE operation that replaces `deleteFile` DataFile newDataFile2 = newDataFile("data_bucket=0"); - DeleteFile newDeleteFile2 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + DeleteFile newDeleteFile2 = newDeletes(dataFile); RowDelta merge = table .newRowDelta() @@ -1556,4 +1546,102 @@ public void testConcurrentMergeRewriteSameDeleteFile() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found new conflicting delete files that can apply"); } + + @TestTemplate + public void testConcurrentDVsForSameDataFile() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + + DataFile dataFile = newDataFile("data_bucket=0"); + commit(table, table.newRowDelta().addRows(dataFile), branch); + + DeleteFile deleteFile1 = newDeletes(dataFile); + RowDelta rowDelta1 = table.newRowDelta().addDeletes(deleteFile1); + + DeleteFile deleteFile2 = newDeletes(dataFile); + RowDelta rowDelta2 = table.newRowDelta().addDeletes(deleteFile2); + + commit(table, rowDelta1, branch); + + assertThatThrownBy(() -> commit(table, rowDelta2, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Found concurrently added DV for %s", dataFile.location()); + } + + @TestTemplate + public void testManifestMergingAfterUpgradeToV3() { + assumeThat(formatVersion).isEqualTo(2); + + // enable manifest merging + table + .updateProperties() + .set(TableProperties.MANIFEST_MERGE_ENABLED, "true") + .set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "2") + .commit(); + + // add a data file + DataFile dataFile = newDataFile("data_bucket=0"); + commit(table, table.newAppend().appendFile(dataFile), branch); + + // commit a delete operation using a positional delete file + DeleteFile deleteFile = newDeleteFileWithRef(dataFile); + assertThat(deleteFile.format()).isEqualTo(FileFormat.PARQUET); + RowDelta rowDelta1 = table.newRowDelta().addDeletes(deleteFile); + Snapshot deleteFileSnapshot = commit(table, rowDelta1, branch); + + // upgrade the table + table.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); + + // commit a DV + DeleteFile dv = newDV(dataFile); + assertThat(dv.format()).isEqualTo(FileFormat.PUFFIN); + RowDelta rowDelta2 = table.newRowDelta().addDeletes(dv); + Snapshot dvSnapshot = commit(table, rowDelta2, branch); + + // both must be part of the table and merged into one manifest + ManifestFile deleteManifest = Iterables.getOnlyElement(dvSnapshot.deleteManifests(table.io())); + validateDeleteManifest( + deleteManifest, + dataSeqs(3L, 2L), + fileSeqs(3L, 2L), + ids(dvSnapshot.snapshotId(), deleteFileSnapshot.snapshotId()), + files(dv, deleteFile), + statuses(Status.ADDED, Status.EXISTING)); + + // only the DV must be assigned during planning + List tasks = planFiles(); + FileScanTask task = Iterables.getOnlyElement(tasks).asFileScanTask(); + assertThat(task.deletes()).hasSize(1); + DeleteFile taskDV = Iterables.getOnlyElement(task.deletes()); + assertThat(taskDV.location()).isEqualTo(dv.location()); + assertThat(taskDV.referencedDataFile()).isEqualTo(dv.referencedDataFile()); + assertThat(taskDV.contentOffset()).isEqualTo(dv.contentOffset()); + assertThat(taskDV.contentSizeInBytes()).isEqualTo(dv.contentSizeInBytes()); + } + + @TestTemplate + public void testInabilityToAddPositionDeleteFilesInTablesWithDVs() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + DeleteFile deleteFile = newDeleteFile(table.spec().specId(), "data_bucket=0"); + assertThatThrownBy(() -> table.newRowDelta().addDeletes(deleteFile)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Must use DVs for position deletes in V%s", formatVersion); + } + + @TestTemplate + public void testInabilityToAddDVToV2Tables() { + assumeThat(formatVersion).isEqualTo(2); + DataFile dataFile = newDataFile("data_bucket=0"); + DeleteFile dv = newDV(dataFile); + assertThatThrownBy(() -> table.newRowDelta().addDeletes(dv)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Must not use DVs for position deletes in V2"); + } + + private List planFiles() { + try (CloseableIterable tasks = table.newBatchScan().useRef(branch).planFiles()) { + return Lists.newArrayList(tasks); + } catch (IOException e) { + throw new RuntimeException(e); + } + } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java index 8a30036f3242..bbe5e8f6cdd8 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java @@ -123,7 +123,7 @@ public void testCachedDeleteFiles() { int specId = table.spec().specId(); DataFile secondSnapshotDataFile = newDataFile("data_bucket=8/data_trunc_2=aa"); - DeleteFile secondSnapshotDeleteFile = newDeleteFile(specId, "data_bucket=8/data_trunc_2=aa"); + DeleteFile secondSnapshotDeleteFile = newDeletes(secondSnapshotDataFile); table .newRowDelta() @@ -131,7 +131,7 @@ public void testCachedDeleteFiles() { .addDeletes(secondSnapshotDeleteFile) .commit(); - DeleteFile thirdSnapshotDeleteFile = newDeleteFile(specId, "data_bucket=8/data_trunc_2=aa"); + DeleteFile thirdSnapshotDeleteFile = newDeletes(secondSnapshotDataFile); ImmutableSet replacedDeleteFiles = ImmutableSet.of(secondSnapshotDeleteFile); ImmutableSet newDeleteFiles = ImmutableSet.of(thirdSnapshotDeleteFile); @@ -248,11 +248,9 @@ public void testSequenceNumbersInAddedDeleteFiles() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - int specId = table.spec().specId(); - - runAddedDeleteFileSequenceNumberTest(newDeleteFile(specId, "data_bucket=8"), 2); + runAddedDeleteFileSequenceNumberTest(newDeletes(FILE_A), 2); - runAddedDeleteFileSequenceNumberTest(newDeleteFile(specId, "data_bucket=28"), 3); + runAddedDeleteFileSequenceNumberTest(newDeletes(FILE_B), 3); } private void runAddedDeleteFileSequenceNumberTest( diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java index b0b9d003e35b..9c67e766a993 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java @@ -78,7 +78,7 @@ public void testFileSizeSummary() { @TestTemplate public void testFileSizeSummaryWithDeletes() { - assumeThat(formatVersion).isGreaterThan(1); + assumeThat(formatVersion).isEqualTo(2); table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); @@ -260,7 +260,7 @@ public void rowDeltaWithDuplicates() { @TestTemplate public void rowDeltaWithDeletesAndDuplicates() { - assumeThat(formatVersion).isGreaterThan(1); + assumeThat(formatVersion).isEqualTo(2); assertThat(listManifestFiles()).isEmpty(); table @@ -325,7 +325,7 @@ public void rewriteWithDuplicateFiles() { @TestTemplate public void rewriteWithDeletesAndDuplicates() { - assumeThat(formatVersion).isGreaterThan(1); + assumeThat(formatVersion).isEqualTo(2); assertThat(listManifestFiles()).isEmpty(); table.newRowDelta().addRows(FILE_A2).addDeletes(FILE_A_DELETES).commit(); diff --git a/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java b/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java index 23e0090ca49f..4e50ee57db41 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestDVWriters.java @@ -34,6 +34,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.BaseDeleteLoader; @@ -295,9 +296,13 @@ public void testApplyPartitionScopedPositionDeletes() throws IOException { } private void commit(DeleteWriteResult result) { + Snapshot startSnapshot = table.currentSnapshot(); RowDelta rowDelta = table.newRowDelta(); result.rewrittenDeleteFiles().forEach(rowDelta::removeDeletes); result.deleteFiles().forEach(rowDelta::addDeletes); + if (startSnapshot != null) { + rowDelta.validateFromSnapshot(startSnapshot.snapshotId()); + } rowDelta.commit(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java index 9361c63176e0..659507e4c5e3 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java @@ -42,7 +42,11 @@ public static List parameters() { new Object[] {2, LOCAL, LOCAL}, new Object[] {2, LOCAL, DISTRIBUTED}, new Object[] {2, DISTRIBUTED, LOCAL}, - new Object[] {2, LOCAL, DISTRIBUTED}); + new Object[] {2, LOCAL, DISTRIBUTED}, + new Object[] {3, LOCAL, LOCAL}, + new Object[] {3, LOCAL, DISTRIBUTED}, + new Object[] {3, DISTRIBUTED, LOCAL}, + new Object[] {3, DISTRIBUTED, DISTRIBUTED}); } private static SparkSession spark = null; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java index acd4688440d1..2665d7ba8d3b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java @@ -41,7 +41,11 @@ public static List parameters() { new Object[] {2, LOCAL, LOCAL}, new Object[] {2, LOCAL, DISTRIBUTED}, new Object[] {2, DISTRIBUTED, LOCAL}, - new Object[] {2, DISTRIBUTED, DISTRIBUTED}); + new Object[] {2, DISTRIBUTED, DISTRIBUTED}, + new Object[] {3, LOCAL, LOCAL}, + new Object[] {3, LOCAL, DISTRIBUTED}, + new Object[] {3, DISTRIBUTED, LOCAL}, + new Object[] {3, DISTRIBUTED, DISTRIBUTED}); } private static SparkSession spark = null; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 79e48f47f241..11d61e599eba 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -719,7 +719,7 @@ public void testRewriteLargeManifestsEvolvedUnpartitionedV1Table() throws IOExce @TestTemplate public void testRewriteSmallDeleteManifestsNonPartitionedTable() throws IOException { - assumeThat(formatVersion).isGreaterThan(1); + assumeThat(formatVersion).isEqualTo(2); PartitionSpec spec = PartitionSpec.unpartitioned(); Map options = Maps.newHashMap(); @@ -792,7 +792,7 @@ public void testRewriteSmallDeleteManifestsNonPartitionedTable() throws IOExcept @TestTemplate public void testRewriteSmallDeleteManifestsPartitionedTable() throws IOException { - assumeThat(formatVersion).isGreaterThan(1); + assumeThat(formatVersion).isEqualTo(2); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); Map options = Maps.newHashMap(); @@ -895,7 +895,7 @@ public void testRewriteSmallDeleteManifestsPartitionedTable() throws IOException @TestTemplate public void testRewriteLargeDeleteManifestsPartitionedTable() throws IOException { - assumeThat(formatVersion).isGreaterThan(1); + assumeThat(formatVersion).isEqualTo(2); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); Map options = Maps.newHashMap(); @@ -956,6 +956,62 @@ public void testRewriteLargeDeleteManifestsPartitionedTable() throws IOException assertThat(deleteManifests).hasSizeGreaterThanOrEqualTo(2); } + @TestTemplate + public void testRewriteManifestsAfterUpgradeToV3() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + Map options = ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + DataFile dataFile1 = newDataFile(table, "c1=1"); + DeleteFile deleteFile1 = newDeletes(table, dataFile1); + table.newRowDelta().addRows(dataFile1).addDeletes(deleteFile1).commit(); + + DataFile dataFile2 = newDataFile(table, "c1=1"); + DeleteFile deleteFile2 = newDeletes(table, dataFile2); + table.newRowDelta().addRows(dataFile2).addDeletes(deleteFile2).commit(); + + // upgrade the table to enable DVs + table.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); + + DataFile dataFile3 = newDataFile(table, "c1=1"); + DeleteFile dv3 = newDV(table, dataFile3); + table.newRowDelta().addRows(dataFile3).addDeletes(dv3).commit(); + + SparkActions actions = SparkActions.get(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + assertThat(result.rewrittenManifests()).as("Action should rewrite 6 manifests").hasSize(6); + assertThat(result.addedManifests()).as("Action should add 2 manifests").hasSize(2); + assertManifestsLocation(result.addedManifests()); + + table.refresh(); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask fileTask : tasks) { + DataFile dataFile = fileTask.file(); + DeleteFile deleteFile = Iterables.getOnlyElement(fileTask.deletes()); + if (dataFile.location().equals(dataFile1.location())) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile1.referencedDataFile()); + assertEqual(deleteFile, deleteFile1); + } else if (dataFile.location().equals(dataFile2.location())) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile2.referencedDataFile()); + assertEqual(deleteFile, deleteFile2); + } else { + assertThat(deleteFile.referencedDataFile()).isEqualTo(dv3.referencedDataFile()); + assertEqual(deleteFile, dv3); + } + } + } + } + private List actualRecords() { return spark .read() From 11d21b26ecbb30361b2b2eee0c335d6cd9560c8d Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Mon, 11 Nov 2024 18:55:42 -0500 Subject: [PATCH 130/313] Kafka Connect: fix Hadoop dependency exclusion (#11516) --- kafka-connect/build.gradle | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index d38d01768e66..15bf013f28b2 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -73,6 +73,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { force 'org.xerial.snappy:snappy-java:1.1.10.7' force 'org.apache.commons:commons-compress:1.27.1' force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.3.0' + force 'com.fasterxml.woodstox:woodstox-core:6.7.0' } } } @@ -96,7 +97,6 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { exclude group: 'org.slf4j' exclude group: 'ch.qos.reload4j' exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'com.fasterxml.woodstox' exclude group: 'com.google.guava' exclude group: 'com.google.protobuf' exclude group: 'org.apache.curator' @@ -105,7 +105,6 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { exclude group: 'org.apache.hadoop', module: 'hadoop-auth' exclude group: 'org.apache.commons', module: 'commons-configuration2' exclude group: 'org.apache.hadoop.thirdparty', module: 'hadoop-shaded-protobuf_3_7' - exclude group: 'org.codehaus.woodstox' exclude group: 'org.eclipse.jetty' } implementation project(':iceberg-orc') From e3f39972863f891481ad9f5a559ffef093976bd7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 12 Nov 2024 13:23:32 +0100 Subject: [PATCH 131/313] Build: Upgrade to Gradle 8.11.0 (#11521) --- gradle/wrapper/gradle-wrapper.properties | 4 ++-- gradlew | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index fb602ee2af06..82dd18b2043e 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,7 +1,7 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionSha256Sum=31c55713e40233a8303827ceb42ca48a47267a0ad4bab9177123121e71524c26 -distributionUrl=https\://services.gradle.org/distributions/gradle-8.10.2-bin.zip +distributionSha256Sum=57dafb5c2622c6cc08b993c85b7c06956a2f53536432a30ead46166dbca0f1e9 +distributionUrl=https\://services.gradle.org/distributions/gradle-8.11-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 57ebc02755ab..79ec571bc6ae 100755 --- a/gradlew +++ b/gradlew @@ -87,7 +87,7 @@ APP_BASE_NAME=${0##*/} APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit if [ ! -e $APP_HOME/gradle/wrapper/gradle-wrapper.jar ]; then - curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.10.2/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.11.0/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. From 50545781d54adedb6e6f1e753323ccf37ba9030f Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Wed, 13 Nov 2024 00:29:08 +0800 Subject: [PATCH 132/313] Spark 3.5: Iceberg parser should passthrough unsupported procedure to delegate (#11480) --- .../IcebergSparkSqlExtensionsParser.scala | 12 +++- .../extensions/TestCallStatementParser.java | 72 +++++++++++++------ .../TestCherrypickSnapshotProcedure.java | 12 +++- .../TestExpireSnapshotsProcedure.java | 11 ++- .../TestFastForwardBranchProcedure.java | 11 ++- .../TestPublishChangesProcedure.java | 12 +++- .../TestRemoveOrphanFilesProcedure.java | 10 ++- .../TestRewriteDataFilesProcedure.java | 13 ++-- .../TestRewriteManifestsProcedure.java | 11 ++- .../TestRollbackToSnapshotProcedure.java | 12 +++- .../TestRollbackToTimestampProcedure.java | 12 +++- .../TestSetCurrentSnapshotProcedure.java | 12 +++- .../spark/procedures/SparkProcedures.java | 5 ++ 13 files changed, 152 insertions(+), 53 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index 02bd59366c13..30940d9cffc2 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -29,6 +29,7 @@ import org.apache.iceberg.common.DynConstructors import org.apache.iceberg.spark.ExtendedParser import org.apache.iceberg.spark.ExtendedParser.RawOrderField import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.procedures.SparkProcedures import org.apache.iceberg.spark.source.SparkTable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.SparkSession @@ -136,8 +137,11 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI // Strip comments of the form /* ... */. This must come after stripping newlines so that // comments that span multiple lines are caught. .replaceAll("/\\*.*?\\*/", " ") + // Strip backtick then `system`.`ancestors_of` changes to system.ancestors_of + .replaceAll("`", "") .trim() - normalized.startsWith("call") || ( + + isIcebergProcedure(normalized) || ( normalized.startsWith("alter table") && ( normalized.contains("add partition field") || normalized.contains("drop partition field") || @@ -151,6 +155,12 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI isSnapshotRefDdl(normalized))) } + // All builtin Iceberg procedures are under the 'system' namespace + private def isIcebergProcedure(normalized: String): Boolean = { + normalized.startsWith("call") && + SparkProcedures.names().asScala.map("system." + _).exists(normalized.contains) + } + private def isSnapshotRefDdl(normalized: String): Boolean = { normalized.contains("create branch") || normalized.contains("replace branch") || diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java index 65a36903735c..ade19de36fe9 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java @@ -68,11 +68,34 @@ public static void stopSpark() { currentSpark.stop(); } + @Test + public void testDelegateUnsupportedProcedure() { + assertThatThrownBy(() -> parser.parsePlan("CALL cat.d.t()")) + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); + } + + @Test + public void testCallWithBackticks() throws ParseException { + CallStatement call = + (CallStatement) parser.parsePlan("CALL cat.`system`.`rollback_to_snapshot`()"); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "rollback_to_snapshot"); + + assertThat(seqAsJavaList(call.args())).hasSize(0); + } + @Test public void testCallWithPositionalArgs() throws ParseException { CallStatement call = - (CallStatement) parser.parsePlan("CALL c.n.func(1, '2', 3L, true, 1.0D, 9.0e1, 900e-1BD)"); - assertThat(seqAsJavaList(call.name())).containsExactly("c", "n", "func"); + (CallStatement) + parser.parsePlan( + "CALL c.system.rollback_to_snapshot(1, '2', 3L, true, 1.0D, 9.0e1, 900e-1BD)"); + assertThat(seqAsJavaList(call.name())).containsExactly("c", "system", "rollback_to_snapshot"); assertThat(seqAsJavaList(call.args())).hasSize(7); @@ -88,8 +111,10 @@ public void testCallWithPositionalArgs() throws ParseException { @Test public void testCallWithNamedArgs() throws ParseException { CallStatement call = - (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, c2 => '2', c3 => true)"); - assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); + (CallStatement) + parser.parsePlan( + "CALL cat.system.rollback_to_snapshot(c1 => 1, c2 => '2', c3 => true)"); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "rollback_to_snapshot"); assertThat(seqAsJavaList(call.args())).hasSize(3); @@ -100,8 +125,9 @@ public void testCallWithNamedArgs() throws ParseException { @Test public void testCallWithMixedArgs() throws ParseException { - CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, '2')"); - assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); + CallStatement call = + (CallStatement) parser.parsePlan("CALL cat.system.rollback_to_snapshot(c1 => 1, '2')"); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "rollback_to_snapshot"); assertThat(seqAsJavaList(call.args())).hasSize(2); @@ -113,8 +139,9 @@ public void testCallWithMixedArgs() throws ParseException { public void testCallWithTimestampArg() throws ParseException { CallStatement call = (CallStatement) - parser.parsePlan("CALL cat.system.func(TIMESTAMP '2017-02-03T10:37:30.00Z')"); - assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); + parser.parsePlan( + "CALL cat.system.rollback_to_snapshot(TIMESTAMP '2017-02-03T10:37:30.00Z')"); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "rollback_to_snapshot"); assertThat(seqAsJavaList(call.args())).hasSize(1); @@ -125,8 +152,9 @@ public void testCallWithTimestampArg() throws ParseException { @Test public void testCallWithVarSubstitution() throws ParseException { CallStatement call = - (CallStatement) parser.parsePlan("CALL cat.system.func('${spark.extra.prop}')"); - assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); + (CallStatement) + parser.parsePlan("CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')"); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "rollback_to_snapshot"); assertThat(seqAsJavaList(call.args())).hasSize(1); @@ -135,28 +163,30 @@ public void testCallWithVarSubstitution() throws ParseException { @Test public void testCallParseError() { - assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) + assertThatThrownBy(() -> parser.parsePlan("CALL cat.system.rollback_to_snapshot kebab")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("missing '(' at 'radish'"); + .hasMessageContaining("missing '(' at 'kebab'"); } @Test public void testCallStripsComments() throws ParseException { List callStatementsWithComments = Lists.newArrayList( - "/* bracketed comment */ CALL cat.system.func('${spark.extra.prop}')", - "/**/ CALL cat.system.func('${spark.extra.prop}')", - "-- single line comment \n CALL cat.system.func('${spark.extra.prop}')", - "-- multiple \n-- single line \n-- comments \n CALL cat.system.func('${spark.extra.prop}')", - "/* select * from multiline_comment \n where x like '%sql%'; */ CALL cat.system.func('${spark.extra.prop}')", + "/* bracketed comment */ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "/**/ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "-- single line comment \n CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "-- multiple \n-- single line \n-- comments \n CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "/* select * from multiline_comment \n where x like '%sql%'; */ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", "/* {\"app\": \"dbt\", \"dbt_version\": \"1.0.1\", \"profile_name\": \"profile1\", \"target_name\": \"dev\", " - + "\"node_id\": \"model.profile1.stg_users\"} \n*/ CALL cat.system.func('${spark.extra.prop}')", + + "\"node_id\": \"model.profile1.stg_users\"} \n*/ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", "/* Some multi-line comment \n" - + "*/ CALL /* inline comment */ cat.system.func('${spark.extra.prop}') -- ending comment", - "CALL -- a line ending comment\n" + "cat.system.func('${spark.extra.prop}')"); + + "*/ CALL /* inline comment */ cat.system.rollback_to_snapshot('${spark.extra.prop}') -- ending comment", + "CALL -- a line ending comment\n" + + "cat.system.rollback_to_snapshot('${spark.extra.prop}')"); for (String sqlText : callStatementsWithComments) { CallStatement call = (CallStatement) parser.parsePlan(sqlText); - assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); + assertThat(seqAsJavaList(call.name())) + .containsExactly("cat", "system", "rollback_to_snapshot"); assertThat(seqAsJavaList(call.args())).hasSize(1); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java index d61456fa738b..08b0754df43d 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import java.util.List; import org.apache.iceberg.Snapshot; @@ -30,7 +31,7 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -169,8 +170,13 @@ public void testInvalidCherrypickSnapshotCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.cherrypick_snapshot not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 970fcc47b80c..0f24c5613f17 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -51,7 +51,7 @@ import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -168,8 +168,13 @@ public void testInvalidExpireSnapshotsCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.expire_snapshots not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java index fd8ee7d91bdc..7eb334f70aa2 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java @@ -29,7 +29,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -171,8 +171,13 @@ public void testInvalidFastForwardBranchCases() { assertThatThrownBy( () -> sql("CALL %s.custom.fast_forward('test_table', 'main', 'newBranch')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.fast_forward not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.fast_forward('test_table', 'main')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java index cfca0c068c19..6284d88a1550 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import java.util.List; import org.apache.iceberg.Snapshot; @@ -30,7 +31,7 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -167,8 +168,13 @@ public void testInvalidApplyWapChangesCases() { assertThatThrownBy( () -> sql("CALL %s.custom.publish_changes('n', 't', 'not_valid')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.publish_changes not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 042b846cb15a..d8feaa77079b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -62,7 +62,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.jupiter.api.AfterEach; @@ -252,8 +251,13 @@ public void testInvalidRemoveOrphanFilesCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.remove_orphan_files not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 4a20521c48bb..93198825e326 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -45,7 +45,7 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.internal.SQLConf; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -694,11 +694,16 @@ public void testInvalidCasesForRewriteDataFiles() { assertThatThrownBy( () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) - .hasMessage("Named and positional arguments cannot be mixed"); + .hasMessageContaining("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.rewrite_data_files not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 83ec1ef84dfc..5eebd9aeb711 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -31,8 +31,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -293,8 +293,13 @@ public void testInvalidRewriteManifestsCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.rewrite_manifests not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java index 315f6c4c1b2b..43df78bf766d 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; @@ -30,7 +31,7 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -254,8 +255,13 @@ public void testInvalidRollbackToSnapshotCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.rollback_to_snapshot not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java index 6b74391898e0..ae35b9f1817c 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.sql.Timestamp; @@ -31,7 +32,7 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -295,8 +296,13 @@ public void testInvalidRollbackToTimestampCases() { assertThatThrownBy( () -> sql("CALL %s.custom.rollback_to_timestamp('n', 't', %s)", catalogName, timestamp)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.rollback_to_timestamp not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java index 1133d74e668a..4c34edef5d25 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; @@ -30,7 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.spark.sql.AnalysisException; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -203,8 +204,13 @@ public void testInvalidRollbackToSnapshotCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.set_current_snapshot not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()).isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")).isEqualTo("'CALL'"); + }); assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) .isInstanceOf(IllegalArgumentException.class) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java index b324cd4422b3..42003b24e94c 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java @@ -20,6 +20,7 @@ import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.function.Supplier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -37,6 +38,10 @@ public static ProcedureBuilder newBuilder(String name) { return builderSupplier != null ? builderSupplier.get() : null; } + public static Set names() { + return BUILDERS.keySet(); + } + private static Map> initProcedureBuilders() { ImmutableMap.Builder> mapBuilder = ImmutableMap.builder(); mapBuilder.put("rollback_to_snapshot", RollbackToSnapshotProcedure::builder); From 4a3817bc788701973899765ddbdf8b0768c9318a Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Tue, 12 Nov 2024 14:30:21 -0500 Subject: [PATCH 133/313] Release: Use `dist/release` KEYS (#11526) * use `dist/release` * use downloads --- dev/source-release.sh | 2 +- site/docs/how-to-release.md | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dev/source-release.sh b/dev/source-release.sh index 8bdec693e2c1..b8f3f410e9d4 100755 --- a/dev/source-release.sh +++ b/dev/source-release.sh @@ -149,7 +149,7 @@ The release tarball, signature, and checksums are here: * https://dist.apache.org/repos/dist/dev/iceberg/apache-iceberg-${version}-rc${rc} You can find the KEYS file here: -* https://dist.apache.org/repos/dist/dev/iceberg/KEYS +* https://downloads.apache.org/iceberg/KEYS Convenience binary artifacts are staged on Nexus. The Maven repository URL is: * https://repository.apache.org/content/repositories/orgapacheiceberg-/ diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index be17d9495d5b..56eb18321697 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -35,12 +35,12 @@ This page describes the procedures that the release manager and voting PMC membe To create a release candidate, you will need: * Apache LDAP credentials for Nexus and SVN -* A [GPG key for signing](https://www.apache.org/dev/release-signing#generate), published in [KEYS](https://dist.apache.org/repos/dist/dev/iceberg/KEYS) +* A [GPG key for signing](https://www.apache.org/dev/release-signing#generate), published in [KEYS](https://downloads.apache.org/iceberg/KEYS) If you have not published your GPG key yet, you must publish it before sending the vote email by doing: ```shell -svn co https://dist.apache.org/repos/dist/dev/iceberg icebergsvn +svn co https://dist.apache.org/repos/dist/release/iceberg icebergsvn cd icebergsvn echo "" >> KEYS # append a newline gpg --list-sigs >> KEYS # append signatures @@ -204,7 +204,7 @@ The release tarball, signature, and checksums are here: * https://dist.apache.org/repos/dist/dev/iceberg/apache-iceberg--rc/ You can find the KEYS file here: -* https://dist.apache.org/repos/dist/dev/iceberg/KEYS +* https://downloads.apache.org/iceberg/KEYS Convenience binary artifacts are staged in Nexus. The Maven repository URL is: * https://repository.apache.org/content/repositories/orgapacheiceberg-/ @@ -349,7 +349,7 @@ verify signatures, checksums, and documentation. First, import the keys. ```bash -curl https://dist.apache.org/repos/dist/dev/iceberg/KEYS -o KEYS +curl https://downloads.apache.org/iceberg/KEYS -o KEYS gpg --import KEYS ``` From 0280885ac95bdf763556a84bb9d7c6fd9c8c5e2a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Wed, 13 Nov 2024 16:00:08 +0100 Subject: [PATCH 134/313] Pig: Remove iceberg-pig (#11380) --- .github/labeler.yml | 6 - .github/workflows/delta-conversion-ci.yml | 1 - .github/workflows/flink-ci.yml | 1 - .github/workflows/hive-ci.yml | 1 - .github/workflows/kafka-connect-ci.yml | 1 - .github/workflows/spark-ci.yml | 1 - README.md | 1 - build.gradle | 33 -- docs/docs/api.md | 1 - gradle/libs.versions.toml | 2 - .../apache/iceberg/mr/InputFormatConfig.java | 6 - .../mr/mapreduce/IcebergInputFormat.java | 13 +- .../iceberg/mr/TestIcebergInputFormats.java | 2 - .../iceberg/pig/IcebergPigInputFormat.java | 308 ------------ .../apache/iceberg/pig/IcebergStorage.java | 348 ------------- .../apache/iceberg/pig/PigParquetReader.java | 462 ------------------ .../org/apache/iceberg/pig/SchemaUtil.java | 171 ------- .../apache/iceberg/pig/SchemaUtilTest.java | 287 ----------- settings.gradle | 2 - site/docs/contribute.md | 1 - 20 files changed, 1 insertion(+), 1647 deletions(-) delete mode 100644 pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java delete mode 100644 pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java delete mode 100644 pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java delete mode 100644 pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java delete mode 100644 pig/src/test/java/org/apache/iceberg/pig/SchemaUtilTest.java diff --git a/.github/labeler.yml b/.github/labeler.yml index d11c68264cb5..6afc3141ee31 100644 --- a/.github/labeler.yml +++ b/.github/labeler.yml @@ -130,12 +130,6 @@ MR: 'mr/**/*' ] -PIG: - - changed-files: - - any-glob-to-any-file: [ - 'pig/**/*' - ] - AWS: - changed-files: - any-glob-to-any-file: [ diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index 9326d9d533fd..521d061f6552 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -53,7 +53,6 @@ on: - 'hive-runtime/**' - 'flink/**' - 'kafka-connect/**' - - 'pig/**' - 'docs/**' - 'site/**' - 'open-api/**' diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 8ed555847861..22f4f008a215 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -53,7 +53,6 @@ on: - 'hive-runtime/**' - 'kafka-connect/**' - 'spark/**' - - 'pig/**' - 'docs/**' - 'site/**' - 'open-api/**' diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index bcaf62cc07f8..d95ca1bd5c6a 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -51,7 +51,6 @@ on: - 'spark/**' - 'flink/**' - 'kafka-connect/**' - - 'pig/**' - 'docs/**' - 'site/**' - 'open-api/**' diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index 98ec18a77953..60cd9188b61d 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -53,7 +53,6 @@ on: - 'hive3-orc-bundle/**' - 'hive-runtime/**' - 'spark/**' - - 'pig/**' - 'docs/**' - 'site/**' - 'open-api/**' diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index b5d91d3cc76c..0d7bd2d3d3e7 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -54,7 +54,6 @@ on: - 'hive-runtime/**' - 'flink/**' - 'kafka-connect/**' - - 'pig/**' - 'docs/**' - 'open-api/**' - 'format/**' diff --git a/README.md b/README.md index 7d2056077804..5c6e5fd96d35 100644 --- a/README.md +++ b/README.md @@ -74,7 +74,6 @@ Iceberg also has modules for adding Iceberg support to processing engines: * `iceberg-spark` is an implementation of Spark's Datasource V2 API for Iceberg with submodules for each spark versions (use runtime jars for a shaded version) * `iceberg-flink` contains classes for integrating with Apache Flink (use iceberg-flink-runtime for a shaded version) * `iceberg-mr` contains an InputFormat and other classes for integrating with Apache Hive -* `iceberg-pig` is an implementation of Pig's LoadFunc API for Iceberg --- **NOTE** diff --git a/build.gradle b/build.gradle index 7990ffbadcb2..81daf14a357f 100644 --- a/build.gradle +++ b/build.gradle @@ -838,39 +838,6 @@ project(':iceberg-arrow') { } } -project(':iceberg-pig') { - test { - useJUnitPlatform() - } - - dependencies { - implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') - api project(':iceberg-api') - implementation project(':iceberg-common') - implementation project(':iceberg-core') - implementation project(':iceberg-parquet') - - implementation(libs.parquet.avro) { - exclude group: 'org.apache.avro', module: 'avro' - // already shaded by Parquet - exclude group: 'it.unimi.dsi' - exclude group: 'org.codehaus.jackson' - } - - compileOnly(libs.pig) { - exclude group: "junit", module: "junit" - } - compileOnly(libs.hadoop2.mapreduce.client.core) - compileOnly(libs.hadoop2.client) { - exclude group: 'org.apache.avro', module: 'avro' - } - - testImplementation(libs.hadoop2.minicluster) { - exclude group: 'org.apache.avro', module: 'avro' - } - } -} - project(':iceberg-nessie') { test { useJUnitPlatform() diff --git a/docs/docs/api.md b/docs/docs/api.md index 286f7bd2254d..e4ea1b1043b4 100644 --- a/docs/docs/api.md +++ b/docs/docs/api.md @@ -251,6 +251,5 @@ This project Iceberg also has modules for adding Iceberg support to processing e * `iceberg-mr` is an implementation of MapReduce and Hive InputFormats and SerDes for Iceberg (use iceberg-hive-runtime for a shaded version for use with Hive) * `iceberg-nessie` is a module used to integrate Iceberg table metadata history and operations with [Project Nessie](https://projectnessie.org/) * `iceberg-data` is a client library used to read Iceberg tables from JVM applications -* `iceberg-pig` is an implementation of Pig's LoadFunc API for Iceberg * `iceberg-runtime` generates a shaded runtime jar for Spark to integrate with iceberg tables diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 85fecabcae1a..57d1810bd23c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -76,7 +76,6 @@ netty-buffer-compat = "4.1.114.Final" object-client-bundle = "3.3.2" orc = "1.9.4" parquet = "1.13.1" -pig = "0.17.0" roaringbitmap = "1.3.0" scala-collection-compat = "2.12.0" slf4j = "2.0.16" @@ -169,7 +168,6 @@ orc-core = { module = "org.apache.orc:orc-core", version.ref = "orc" } parquet-avro = { module = "org.apache.parquet:parquet-avro", version.ref = "parquet" } parquet-column = { module = "org.apache.parquet:parquet-column", version.ref = "parquet" } parquet-hadoop = { module = "org.apache.parquet:parquet-hadoop", version.ref = "parquet" } -pig = { module = "org.apache.pig:pig", version.ref = "pig" } roaringbitmap = { module = "org.roaringbitmap:RoaringBitmap", version.ref = "roaringbitmap" } scala-collection-compat = { module = "org.scala-lang.modules:scala-collection-compat_2.13", version.ref = "scala-collection-compat"} slf4j-api = { module = "org.slf4j:slf4j-api", version.ref = "slf4j" } diff --git a/mr/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java b/mr/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java index 185617aec258..415eb8c9b858 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java +++ b/mr/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java @@ -77,7 +77,6 @@ private InputFormatConfig() {} public static final String CATALOG_CONFIG_PREFIX = "iceberg.catalog."; public enum InMemoryDataModel { - PIG, HIVE, GENERIC // Default data model is of Iceberg Generics } @@ -169,11 +168,6 @@ public ConfigBuilder useHiveRows() { return this; } - public ConfigBuilder usePigTuples() { - conf.set(IN_MEMORY_DATA_MODEL, InMemoryDataModel.PIG.name()); - return this; - } - /** * Compute platforms pass down filters to data sources. If the data source cannot apply some * filters, or only partially applies the filter, it will return the residual filter back. If diff --git a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index 7ea2d26891f8..9b8d4e9247a2 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -166,9 +166,7 @@ private List planInputSplits( Table serializableTable = SerializableTable.copyOf(table); tasksIterable.forEach( task -> { - if (applyResidual - && (model == InputFormatConfig.InMemoryDataModel.HIVE - || model == InputFormatConfig.InMemoryDataModel.PIG)) { + if (applyResidual && (model == InputFormatConfig.InMemoryDataModel.HIVE)) { // TODO: We do not support residual evaluation for HIVE and PIG in memory data model // yet checkResiduals(task); @@ -347,9 +345,6 @@ private CloseableIterable openTask(FileScanTask currentTask, Schema readSchem @SuppressWarnings("unchecked") private CloseableIterable open(FileScanTask currentTask, Schema readSchema) { switch (inMemoryDataModel) { - case PIG: - // TODO: Support Pig and Hive object models for IcebergInputFormat - throw new UnsupportedOperationException("Pig and Hive object models are not supported."); case HIVE: return openTask(currentTask, readSchema); case GENERIC: @@ -390,7 +385,6 @@ private CloseableIterable newAvroIterable( } switch (inMemoryDataModel) { - case PIG: case HIVE: // TODO implement value readers for Pig and Hive throw new UnsupportedOperationException( @@ -413,8 +407,6 @@ private CloseableIterable newParquetIterable( CloseableIterable parquetIterator = null; switch (inMemoryDataModel) { - case PIG: - throw new UnsupportedOperationException("Parquet support not yet supported for Pig"); case HIVE: if (HiveVersion.min(HiveVersion.HIVE_3)) { parquetIterator = @@ -459,9 +451,6 @@ private CloseableIterable newOrcIterable( CloseableIterable orcIterator = null; // ORC does not support reuse containers yet switch (inMemoryDataModel) { - case PIG: - // TODO: implement value readers for Pig - throw new UnsupportedOperationException("ORC support not yet supported for Pig"); case HIVE: if (HiveVersion.min(HiveVersion.HIVE_3)) { orcIterator = diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java b/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java index 2b93b276ad94..668703cc5d92 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java @@ -223,8 +223,6 @@ public void testFailedResidualFiltering() throws Exception { .hasMessage( "Filter expression ref(name=\"id\") == 0 is not completely satisfied. Additional rows can be returned not satisfied by the filter expression"); - builder.usePigTuples(); - assertThatThrownBy(() -> testInputFormat.create(builder.conf())) .isInstanceOf(UnsupportedOperationException.class) .hasMessage( diff --git a/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java deleted file mode 100644 index 932de72ac8c0..000000000000 --- a/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java +++ /dev/null @@ -1,308 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.pig; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapreduce.InputFormat; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.hadoop.HadoopInputFile; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.ByteBuffers; -import org.apache.iceberg.util.SerializationUtil; -import org.apache.pig.data.DataByteArray; -import org.apache.pig.impl.util.ObjectSerializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @deprecated will be removed in 1.8.0 - */ -@Deprecated -public class IcebergPigInputFormat extends InputFormat { - private static final Logger LOG = LoggerFactory.getLogger(IcebergPigInputFormat.class); - - static final String ICEBERG_SCHEMA = "iceberg.schema"; - static final String ICEBERG_PROJECTED_FIELDS = "iceberg.projected.fields"; - static final String ICEBERG_FILTER_EXPRESSION = "iceberg.filter.expression"; - - private final Table table; - private final String signature; - private List splits; - - IcebergPigInputFormat(Table table, String signature) { - LOG.warn("Iceberg Pig is deprecated and will be removed in Iceberg 1.8.0"); - this.table = table; - this.signature = signature; - } - - @Override - @SuppressWarnings("unchecked") - public List getSplits(JobContext context) throws IOException { - if (splits != null) { - LOG.info("Returning cached splits: {}", splits.size()); - return splits; - } - - splits = Lists.newArrayList(); - - TableScan scan = table.newScan(); - - // Apply Filters - Expression filterExpression = - (Expression) - ObjectSerializer.deserialize( - context.getConfiguration().get(scope(ICEBERG_FILTER_EXPRESSION))); - LOG.info("[{}]: iceberg filter expressions: {}", signature, filterExpression); - - if (filterExpression != null) { - LOG.info("Filter Expression: {}", filterExpression); - scan = scan.filter(filterExpression); - } - - // Wrap in Splits - try (CloseableIterable tasks = scan.planTasks()) { - tasks.forEach(scanTask -> splits.add(new IcebergSplit(scanTask))); - } - - return splits; - } - - @Override - public RecordReader createRecordReader(InputSplit split, TaskAttemptContext context) { - return new IcebergRecordReader<>(); - } - - private static class IcebergSplit extends InputSplit implements Writable { - private static final String[] ANYWHERE = new String[] {"*"}; - - private CombinedScanTask task; - - IcebergSplit(CombinedScanTask task) { - this.task = task; - } - - @Override - public long getLength() { - return task.files().stream().mapToLong(FileScanTask::length).sum(); - } - - @Override - public String[] getLocations() { - return ANYWHERE; - } - - @Override - public void write(DataOutput out) throws IOException { - byte[] data = SerializationUtil.serializeToBytes(this.task); - out.writeInt(data.length); - out.write(data); - } - - @Override - public void readFields(DataInput in) throws IOException { - byte[] data = new byte[in.readInt()]; - in.readFully(data); - - this.task = SerializationUtil.deserializeFromBytes(data); - } - } - - private String scope(String key) { - return key + '.' + signature; - } - - public class IcebergRecordReader extends RecordReader { - private TaskAttemptContext context; - - private Iterator tasks; - - private CloseableIterable reader; - private Iterator recordIterator; - private T currentRecord; - - @Override - public void initialize(InputSplit split, TaskAttemptContext initContext) throws IOException { - this.context = initContext; - - CombinedScanTask task = ((IcebergSplit) split).task; - this.tasks = task.files().iterator(); - - advance(); - } - - @SuppressWarnings("unchecked") - private boolean advance() throws IOException { - if (reader != null) { - reader.close(); - } - - if (!tasks.hasNext()) { - return false; - } - - FileScanTask currentTask = tasks.next(); - - Schema tableSchema = - (Schema) - ObjectSerializer.deserialize(context.getConfiguration().get(scope(ICEBERG_SCHEMA))); - LOG.debug("[{}]: Task table schema: {}", signature, tableSchema); - - List projectedFields = - (List) - ObjectSerializer.deserialize( - context.getConfiguration().get(scope(ICEBERG_PROJECTED_FIELDS))); - LOG.debug("[{}]: Task projected fields: {}", signature, projectedFields); - - Schema projectedSchema = - projectedFields != null ? SchemaUtil.project(tableSchema, projectedFields) : tableSchema; - - PartitionSpec spec = currentTask.asFileScanTask().spec(); - DataFile file = currentTask.file(); - InputFile inputFile = HadoopInputFile.fromLocation(file.path(), context.getConfiguration()); - - Set idColumns = spec.identitySourceIds(); - - // schema needed for the projection and filtering - boolean hasJoinedPartitionColumns = !idColumns.isEmpty(); - - switch (file.format()) { - case PARQUET: - Map partitionValueMap = Maps.newHashMap(); - - if (hasJoinedPartitionColumns) { - - Schema readSchema = TypeUtil.selectNot(projectedSchema, idColumns); - Schema projectedPartitionSchema = TypeUtil.select(projectedSchema, idColumns); - - Map partitionSpecFieldIndexMap = Maps.newHashMap(); - for (int i = 0; i < spec.fields().size(); i++) { - partitionSpecFieldIndexMap.put(spec.fields().get(i).name(), i); - } - - for (Types.NestedField field : projectedPartitionSchema.columns()) { - int partitionIndex = partitionSpecFieldIndexMap.get(field.name()); - - Object partitionValue = file.partition().get(partitionIndex, Object.class); - partitionValueMap.put( - field.fieldId(), convertPartitionValue(field.type(), partitionValue)); - } - - reader = - Parquet.read(inputFile) - .project(readSchema) - .split(currentTask.start(), currentTask.length()) - .filter(currentTask.residual()) - .createReaderFunc( - fileSchema -> - PigParquetReader.buildReader( - fileSchema, projectedSchema, partitionValueMap)) - .build(); - } else { - reader = - Parquet.read(inputFile) - .project(projectedSchema) - .split(currentTask.start(), currentTask.length()) - .filter(currentTask.residual()) - .createReaderFunc( - fileSchema -> - PigParquetReader.buildReader( - fileSchema, projectedSchema, partitionValueMap)) - .build(); - } - - recordIterator = reader.iterator(); - - break; - default: - throw new UnsupportedOperationException("Unsupported file format: " + file.format()); - } - - return true; - } - - private Object convertPartitionValue(Type type, Object value) { - if (type.typeId() == Types.BinaryType.get().typeId()) { - return new DataByteArray(ByteBuffers.toByteArray((ByteBuffer) value)); - } - - return value; - } - - @Override - public boolean nextKeyValue() throws IOException { - if (recordIterator.hasNext()) { - currentRecord = recordIterator.next(); - return true; - } - - while (advance()) { - if (recordIterator.hasNext()) { - currentRecord = recordIterator.next(); - return true; - } - } - - return false; - } - - @Override - public Void getCurrentKey() { - return null; - } - - @Override - public T getCurrentValue() { - return currentRecord; - } - - @Override - public float getProgress() { - return 0; - } - - @Override - public void close() {} - } -} diff --git a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java deleted file mode 100644 index 0ce23c39913a..000000000000 --- a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java +++ /dev/null @@ -1,348 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.pig; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.stream.Collectors; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapreduce.InputFormat; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.util.ReflectionUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.Tables; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.pig.IcebergPigInputFormat.IcebergRecordReader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.NaNUtil; -import org.apache.pig.Expression; -import org.apache.pig.Expression.BetweenExpression; -import org.apache.pig.Expression.BinaryExpression; -import org.apache.pig.Expression.Column; -import org.apache.pig.Expression.Const; -import org.apache.pig.Expression.InExpression; -import org.apache.pig.Expression.OpType; -import org.apache.pig.Expression.UnaryExpression; -import org.apache.pig.LoadFunc; -import org.apache.pig.LoadMetadata; -import org.apache.pig.LoadPredicatePushdown; -import org.apache.pig.LoadPushDown; -import org.apache.pig.ResourceSchema; -import org.apache.pig.ResourceStatistics; -import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit; -import org.apache.pig.data.Tuple; -import org.apache.pig.impl.logicalLayer.FrontendException; -import org.apache.pig.impl.util.ObjectSerializer; -import org.apache.pig.impl.util.UDFContext; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @deprecated will be removed in 1.8.0 - */ -@Deprecated -public class IcebergStorage extends LoadFunc - implements LoadMetadata, LoadPredicatePushdown, LoadPushDown { - private static final Logger LOG = LoggerFactory.getLogger(IcebergStorage.class); - - public static final String PIG_ICEBERG_TABLES_IMPL = "pig.iceberg.tables.impl"; - private static Tables iceberg; - private static final Map TABLES = Maps.newConcurrentMap(); - private static final Map LOCATIONS = Maps.newConcurrentMap(); - - private String signature; - - private IcebergRecordReader reader; - - public IcebergStorage() { - LOG.warn("Iceberg Pig is deprecated and will be removed in Iceberg 1.8.0"); - } - - @Override - public void setLocation(String location, Job job) { - LOG.info("[{}]: setLocation() -> {}", signature, location); - - LOCATIONS.put(signature, location); - - Configuration conf = job.getConfiguration(); - - copyUDFContextToScopedConfiguration(conf, IcebergPigInputFormat.ICEBERG_SCHEMA); - copyUDFContextToScopedConfiguration(conf, IcebergPigInputFormat.ICEBERG_PROJECTED_FIELDS); - copyUDFContextToScopedConfiguration(conf, IcebergPigInputFormat.ICEBERG_FILTER_EXPRESSION); - } - - @Override - public InputFormat getInputFormat() { - LOG.info("[{}]: getInputFormat()", signature); - String location = LOCATIONS.get(signature); - - return new IcebergPigInputFormat(TABLES.get(location), signature); - } - - @Override - public Tuple getNext() throws IOException { - if (!reader.nextKeyValue()) { - return null; - } - - return (Tuple) reader.getCurrentValue(); - } - - @Override - public void prepareToRead(RecordReader newReader, PigSplit split) { - LOG.info("[{}]: prepareToRead() -> {}", signature, split); - - this.reader = (IcebergRecordReader) newReader; - } - - @Override - public ResourceSchema getSchema(String location, Job job) throws IOException { - LOG.info("[{}]: getSchema() -> {}", signature, location); - - Schema schema = load(location, job).schema(); - storeInUDFContext(IcebergPigInputFormat.ICEBERG_SCHEMA, schema); - - return SchemaUtil.convert(schema); - } - - @Override - public ResourceStatistics getStatistics(String location, Job job) { - LOG.info("[{}]: getStatistics() -> : {}", signature, location); - - return null; - } - - @Override - public String[] getPartitionKeys(String location, Job job) { - LOG.info("[{}]: getPartitionKeys()", signature); - return new String[0]; - } - - @Override - public void setPartitionFilter(Expression partitionFilter) { - LOG.info("[{}]: setPartitionFilter() -> {}", signature, partitionFilter); - } - - @Override - public List getPredicateFields(String location, Job job) throws IOException { - LOG.info("[{}]: getPredicateFields() -> {}", signature, location); - Schema schema = load(location, job).schema(); - - List result = Lists.newArrayList(); - - for (Types.NestedField nf : schema.columns()) { - switch (nf.type().typeId()) { - case MAP: - case LIST: - case STRUCT: - continue; - default: - result.add(nf.name()); - } - } - - return result; - } - - @Override - public ImmutableList getSupportedExpressionTypes() { - LOG.info("[{}]: getSupportedExpressionTypes()", signature); - return ImmutableList.of( - OpType.OP_AND, - OpType.OP_OR, - OpType.OP_EQ, - OpType.OP_NE, - OpType.OP_NOT, - OpType.OP_GE, - OpType.OP_GT, - OpType.OP_LE, - OpType.OP_LT, - OpType.OP_BETWEEN, - OpType.OP_IN, - OpType.OP_NULL); - } - - @Override - public void setPushdownPredicate(Expression predicate) throws IOException { - LOG.info("[{}]: setPushdownPredicate()", signature); - LOG.info("[{}]: Pig predicate expression: {}", signature, predicate); - - org.apache.iceberg.expressions.Expression icebergExpression = convert(predicate); - - LOG.info("[{}]: Iceberg predicate expression: {}", signature, icebergExpression); - - storeInUDFContext(IcebergPigInputFormat.ICEBERG_FILTER_EXPRESSION, icebergExpression); - } - - private org.apache.iceberg.expressions.Expression convert(Expression expression) - throws IOException { - OpType op = expression.getOpType(); - - if (expression instanceof BinaryExpression) { - Expression lhs = ((BinaryExpression) expression).getLhs(); - Expression rhs = ((BinaryExpression) expression).getRhs(); - - switch (op) { - case OP_AND: - return Expressions.and(convert(lhs), convert(rhs)); - case OP_OR: - return Expressions.or(convert(lhs), convert(rhs)); - case OP_BETWEEN: - BetweenExpression between = (BetweenExpression) rhs; - return Expressions.and( - convert(OpType.OP_GE, (Column) lhs, (Const) between.getLower()), - convert(OpType.OP_LE, (Column) lhs, (Const) between.getUpper())); - case OP_IN: - return ((InExpression) rhs) - .getValues().stream() - .map(value -> convert(OpType.OP_EQ, (Column) lhs, (Const) value)) - .reduce(Expressions.alwaysFalse(), Expressions::or); - default: - if (lhs instanceof Column && rhs instanceof Const) { - return convert(op, (Column) lhs, (Const) rhs); - } else if (lhs instanceof Const && rhs instanceof Column) { - throw new FrontendException("Invalid expression ordering " + expression); - } - } - - } else if (expression instanceof UnaryExpression) { - Expression unary = ((UnaryExpression) expression).getExpression(); - - switch (op) { - case OP_NOT: - return Expressions.not(convert(unary)); - case OP_NULL: - return Expressions.isNull(((Column) unary).getName()); - default: - throw new FrontendException("Unsupported unary operator" + op); - } - } - - throw new FrontendException("Failed to pushdown expression " + expression); - } - - private org.apache.iceberg.expressions.Expression convert(OpType op, Column col, Const constant) { - String name = col.getName(); - Object value = constant.getValue(); - - switch (op) { - case OP_GE: - return Expressions.greaterThanOrEqual(name, value); - case OP_GT: - return Expressions.greaterThan(name, value); - case OP_LE: - return Expressions.lessThanOrEqual(name, value); - case OP_LT: - return Expressions.lessThan(name, value); - case OP_EQ: - return NaNUtil.isNaN(value) ? Expressions.isNaN(name) : Expressions.equal(name, value); - case OP_NE: - return NaNUtil.isNaN(value) ? Expressions.notNaN(name) : Expressions.notEqual(name, value); - } - - throw new RuntimeException( - String.format( - "[%s]: Failed to pushdown expression: %s %s %s", signature, col, op, constant)); - } - - @Override - public List getFeatures() { - return Collections.singletonList(OperatorSet.PROJECTION); - } - - @Override - public RequiredFieldResponse pushProjection(RequiredFieldList requiredFieldList) { - LOG.info("[{}]: pushProjection() -> {}", signature, requiredFieldList); - - try { - List projection = - requiredFieldList.getFields().stream() - .map(RequiredField::getAlias) - .collect(Collectors.toList()); - - storeInUDFContext(IcebergPigInputFormat.ICEBERG_PROJECTED_FIELDS, (Serializable) projection); - } catch (IOException e) { - throw new RuntimeException(e); - } - - return new RequiredFieldResponse(true); - } - - @Override - public void setUDFContextSignature(String newSignature) { - this.signature = newSignature; - } - - private void storeInUDFContext(String key, Serializable value) throws IOException { - Properties properties = - UDFContext.getUDFContext().getUDFProperties(this.getClass(), new String[] {signature}); - - properties.setProperty(key, ObjectSerializer.serialize(value)); - } - - private void copyUDFContextToScopedConfiguration(Configuration conf, String key) { - String value = - UDFContext.getUDFContext() - .getUDFProperties(this.getClass(), new String[] {signature}) - .getProperty(key); - - if (value != null) { - conf.set(key + '.' + signature, value); - } - } - - @Override - public String relativeToAbsolutePath(String location, Path curDir) throws IOException { - return location; - } - - private Table load(String location, Job job) throws IOException { - if (iceberg == null) { - Class tablesImpl = - job.getConfiguration().getClass(PIG_ICEBERG_TABLES_IMPL, HadoopTables.class); - LOG.info("Initializing iceberg tables implementation: {}", tablesImpl); - iceberg = (Tables) ReflectionUtils.newInstance(tablesImpl, job.getConfiguration()); - } - - Table result = TABLES.get(location); - - if (result == null) { - try { - LOG.info("[{}]: Loading table for location: {}", signature, location); - result = iceberg.load(location); - TABLES.put(location, result); - } catch (Exception e) { - throw new FrontendException("Failed to instantiate tables implementation", e); - } - } - - return result; - } -} diff --git a/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java b/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java deleted file mode 100644 index 15ba9068caf5..000000000000 --- a/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java +++ /dev/null @@ -1,462 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.pig; - -import java.time.Instant; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.parquet.ParquetSchemaUtil; -import org.apache.iceberg.parquet.ParquetValueReader; -import org.apache.iceberg.parquet.ParquetValueReaders; -import org.apache.iceberg.parquet.ParquetValueReaders.BinaryAsDecimalReader; -import org.apache.iceberg.parquet.ParquetValueReaders.FloatAsDoubleReader; -import org.apache.iceberg.parquet.ParquetValueReaders.IntAsLongReader; -import org.apache.iceberg.parquet.ParquetValueReaders.IntegerAsDecimalReader; -import org.apache.iceberg.parquet.ParquetValueReaders.LongAsDecimalReader; -import org.apache.iceberg.parquet.ParquetValueReaders.PrimitiveReader; -import org.apache.iceberg.parquet.ParquetValueReaders.RepeatedKeyValueReader; -import org.apache.iceberg.parquet.ParquetValueReaders.RepeatedReader; -import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; -import org.apache.iceberg.parquet.ParquetValueReaders.StringReader; -import org.apache.iceberg.parquet.ParquetValueReaders.StructReader; -import org.apache.iceberg.parquet.ParquetValueReaders.UnboxedReader; -import org.apache.iceberg.parquet.TypeWithSchemaVisitor; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Type.TypeID; -import org.apache.iceberg.types.Types; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; -import org.apache.pig.backend.executionengine.ExecException; -import org.apache.pig.data.BagFactory; -import org.apache.pig.data.DataBag; -import org.apache.pig.data.DataByteArray; -import org.apache.pig.data.Tuple; -import org.apache.pig.data.TupleFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @deprecated will be removed in 1.8.0 - */ -@Deprecated -public class PigParquetReader { - - private static final Logger LOG = LoggerFactory.getLogger(PigParquetReader.class); - - private PigParquetReader() { - LOG.warn("Iceberg Pig is deprecated and will be removed in Iceberg 1.8.0"); - } - - @SuppressWarnings("unchecked") - public static ParquetValueReader buildReader( - MessageType fileSchema, Schema expectedSchema, Map partitionValues) { - - if (ParquetSchemaUtil.hasIds(fileSchema)) { - return (ParquetValueReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, partitionValues)); - } else { - return (ParquetValueReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), - fileSchema, - new FallbackReadBuilder(fileSchema, partitionValues)); - } - } - - private static class FallbackReadBuilder extends ReadBuilder { - FallbackReadBuilder(MessageType type, Map partitionValues) { - super(type, partitionValues); - } - - @Override - public ParquetValueReader message( - Types.StructType expected, MessageType message, List> fieldReaders) { - // the top level matches by ID, but the remaining IDs are missing - return super.struct(expected, message, fieldReaders); - } - - @Override - public ParquetValueReader struct( - Types.StructType ignored, GroupType struct, List> fieldReaders) { - // the expected struct is ignored because nested fields are never found when the - List> newFields = - Lists.newArrayListWithExpectedSize(fieldReaders.size()); - List types = Lists.newArrayListWithExpectedSize(fieldReaders.size()); - List fields = struct.getFields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i); - int fieldD = getMessageType().getMaxDefinitionLevel(path(fieldType.getName())) - 1; - newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); - types.add(fieldType); - } - - return new TupleReader(types, newFields); - } - } - - private static class ReadBuilder extends TypeWithSchemaVisitor> { - private final MessageType type; - private final Map partitionValues; - - ReadBuilder(MessageType type, Map partitionValues) { - this.type = type; - this.partitionValues = partitionValues; - } - - MessageType getMessageType() { - return this.type; - } - - @Override - public ParquetValueReader message( - Types.StructType expected, MessageType message, List> fieldReaders) { - return struct(expected, message.asGroupType(), fieldReaders); - } - - @Override - public ParquetValueReader struct( - Types.StructType expected, GroupType struct, List> fieldReaders) { - // match the expected struct's order - Map> readersById = Maps.newHashMap(); - Map typesById = Maps.newHashMap(); - Map maxDefinitionLevelsById = Maps.newHashMap(); - List fields = struct.getFields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i); - int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; - int id = fieldType.getId().intValue(); - readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); - typesById.put(id, fieldType); - if (partitionValues.containsKey(id)) { - maxDefinitionLevelsById.put(id, fieldD); - } - } - - List expectedFields = - expected != null ? expected.fields() : ImmutableList.of(); - List> reorderedFields = - Lists.newArrayListWithExpectedSize(expectedFields.size()); - List types = Lists.newArrayListWithExpectedSize(expectedFields.size()); - // Defaulting to parent max definition level - int defaultMaxDefinitionLevel = type.getMaxDefinitionLevel(currentPath()); - for (Types.NestedField field : expectedFields) { - int id = field.fieldId(); - if (partitionValues.containsKey(id)) { - // the value may be null so containsKey is used to check for a partition value - int fieldMaxDefinitionLevel = - maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel); - reorderedFields.add( - ParquetValueReaders.constant(partitionValues.get(id), fieldMaxDefinitionLevel)); - types.add(null); - } else { - ParquetValueReader reader = readersById.get(id); - if (reader != null) { - reorderedFields.add(reader); - types.add(typesById.get(id)); - } else { - reorderedFields.add(ParquetValueReaders.nulls()); - types.add(null); - } - } - } - - return new TupleReader(types, reorderedFields); - } - - @Override - public ParquetValueReader list( - Types.ListType expectedList, GroupType array, ParquetValueReader elementReader) { - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; - int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; - - Type elementType = ParquetSchemaUtil.determineListElementType(array); - int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1; - - return new ArrayReader<>( - repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader)); - } - - @Override - public ParquetValueReader map( - Types.MapType expectedMap, - GroupType map, - ParquetValueReader keyReader, - ParquetValueReader valueReader) { - GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; - int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; - - Type keyType = repeatedKeyValue.getType(0); - int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1; - Type valueType = repeatedKeyValue.getType(1); - int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1; - - return new MapReader<>( - repeatedD, - repeatedR, - ParquetValueReaders.option(keyType, keyD, keyReader), - ParquetValueReaders.option(valueType, valueD, valueReader)); - } - - @Override - public ParquetValueReader primitive( - org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { - ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return new StringReader(desc); - case DATE: - return new DateReader(desc); - case INT_8: - case INT_16: - case INT_32: - if (expected != null && expected.typeId() == Types.LongType.get().typeId()) { - return new IntAsLongReader(desc); - } else { - return new UnboxedReader(desc); - } - case INT_64: - return new UnboxedReader<>(desc); - case TIMESTAMP_MILLIS: - return new TimestampMillisReader(desc); - case TIMESTAMP_MICROS: - return new TimestampMicrosReader(desc); - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return new BinaryAsDecimalReader(desc, decimal.getScale()); - case INT32: - return new IntegerAsDecimalReader(desc, decimal.getScale()); - case INT64: - return new LongAsDecimalReader(desc, decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - default: - throw new UnsupportedOperationException( - "Unsupported type: " + primitive.getOriginalType()); - } - } - - switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - return new BytesReader(desc); - case INT32: - if (expected != null && expected.typeId() == TypeID.LONG) { - return new IntAsLongReader(desc); - } else { - return new UnboxedReader<>(desc); - } - case FLOAT: - if (expected != null && expected.typeId() == TypeID.DOUBLE) { - return new FloatAsDoubleReader(desc); - } else { - return new UnboxedReader<>(desc); - } - case BOOLEAN: - case INT64: - case DOUBLE: - return new UnboxedReader<>(desc); - default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); - } - } - } - - private static class DateReader extends PrimitiveReader { - private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); - - DateReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public String read(String reuse) { - OffsetDateTime day = EPOCH.plusDays(column.nextInteger()); - return String.format( - Locale.ROOT, - "%04d-%02d-%02d", - day.getYear(), - day.getMonth().getValue(), - day.getDayOfMonth()); - } - } - - private static class BytesReader extends PrimitiveReader { - BytesReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public DataByteArray read(DataByteArray reuse) { - byte[] bytes = column.nextBinary().getBytes(); - return new DataByteArray(bytes); - } - } - - private static class TimestampMicrosReader extends UnboxedReader { - private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); - - TimestampMicrosReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public String read(String ignored) { - return ChronoUnit.MICROS.addTo(EPOCH, column.nextLong()).toString(); - } - } - - private static class TimestampMillisReader extends UnboxedReader { - private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); - - TimestampMillisReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public String read(String ignored) { - return ChronoUnit.MILLIS.addTo(EPOCH, column.nextLong()).toString(); - } - } - - private static class MapReader extends RepeatedKeyValueReader, Map, K, V> { - private final ReusableEntry nullEntry = new ReusableEntry<>(); - - MapReader( - int definitionLevel, - int repetitionLevel, - ParquetValueReader keyReader, - ParquetValueReader valueReader) { - super(definitionLevel, repetitionLevel, keyReader, valueReader); - } - - @Override - protected Map newMapData(Map reuse) { - return new LinkedHashMap<>(); - } - - @Override - protected Map.Entry getPair(Map reuse) { - return nullEntry; - } - - @Override - protected void addPair(Map map, K key, V value) { - map.put(key, value); - } - - @Override - protected Map buildMap(Map map) { - return map; - } - } - - private static class ArrayReader extends RepeatedReader { - private final BagFactory bagFactory = BagFactory.getInstance(); - private final TupleFactory tupleFactory = TupleFactory.getInstance(); - - ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader reader) { - super(definitionLevel, repetitionLevel, reader); - } - - @Override - protected DataBag newListData(DataBag reuse) { - return bagFactory.newDefaultBag(); - } - - @Override - protected T getElement(DataBag list) { - return null; - } - - @Override - protected void addElement(DataBag bag, T element) { - bag.add(tupleFactory.newTuple(element)); - } - - @Override - protected DataBag buildList(DataBag bag) { - return bag; - } - } - - private static class TupleReader extends StructReader { - private static final TupleFactory TF = TupleFactory.getInstance(); - private final int numColumns; - - TupleReader(List types, List> readers) { - super(types, readers); - this.numColumns = readers.size(); - } - - @Override - protected Tuple newStructData(Tuple reuse) { - return TF.newTuple(numColumns); - } - - @Override - protected Object getField(Tuple tuple, int pos) { - return null; - } - - @Override - protected Tuple buildStruct(Tuple tuple) { - return tuple; - } - - @Override - protected void set(Tuple tuple, int pos, Object value) { - try { - tuple.set(pos, value); - } catch (ExecException e) { - throw new RuntimeException( - String.format( - Locale.ROOT, "Error setting tuple value for pos: %d, value: %s", pos, value), - e); - } - } - } -} diff --git a/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java b/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java deleted file mode 100644 index 4602a5effa97..000000000000 --- a/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.pig; - -import java.io.IOException; -import java.util.List; -import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.pig.ResourceSchema; -import org.apache.pig.ResourceSchema.ResourceFieldSchema; -import org.apache.pig.data.DataType; -import org.apache.pig.impl.logicalLayer.FrontendException; - -/** - * @deprecated will be removed in 1.8.0 - */ -@Deprecated -public class SchemaUtil { - - private SchemaUtil() {} - - public static ResourceSchema convert(Schema icebergSchema) throws IOException { - ResourceSchema result = new ResourceSchema(); - result.setFields(convertFields(icebergSchema.columns())); - return result; - } - - private static ResourceFieldSchema convert(Types.NestedField field) throws IOException { - ResourceFieldSchema result = convert(field.type()); - result.setName(field.name()); - result.setDescription(String.format("FieldId: %s", field.fieldId())); - - return result; - } - - private static ResourceFieldSchema convert(Type type) throws IOException { - ResourceFieldSchema result = new ResourceFieldSchema(); - result.setType(convertType(type)); - - if (!type.isPrimitiveType()) { - result.setSchema(convertComplex(type)); - } - - return result; - } - - private static ResourceFieldSchema[] convertFields(List fields) - throws IOException { - List result = Lists.newArrayList(); - - for (Types.NestedField nf : fields) { - result.add(convert(nf)); - } - - return result.toArray(new ResourceFieldSchema[0]); - } - - private static byte convertType(Type type) throws IOException { - switch (type.typeId()) { - case BOOLEAN: - return DataType.BOOLEAN; - case INTEGER: - return DataType.INTEGER; - case LONG: - return DataType.LONG; - case FLOAT: - return DataType.FLOAT; - case DOUBLE: - return DataType.DOUBLE; - case TIMESTAMP: - return DataType.CHARARRAY; - case DATE: - return DataType.CHARARRAY; - case STRING: - return DataType.CHARARRAY; - case FIXED: - return DataType.BYTEARRAY; - case BINARY: - return DataType.BYTEARRAY; - case DECIMAL: - return DataType.BIGDECIMAL; - case STRUCT: - return DataType.TUPLE; - case LIST: - return DataType.BAG; - case MAP: - return DataType.MAP; - default: - throw new FrontendException("Unsupported primitive type:" + type); - } - } - - private static ResourceSchema convertComplex(Type type) throws IOException { - ResourceSchema result = new ResourceSchema(); - - switch (type.typeId()) { - case STRUCT: - Types.StructType structType = type.asStructType(); - - List fields = Lists.newArrayList(); - - for (Types.NestedField f : structType.fields()) { - fields.add(convert(f)); - } - - result.setFields(fields.toArray(new ResourceFieldSchema[0])); - - return result; - case LIST: - Types.ListType listType = type.asListType(); - - ResourceFieldSchema[] elementFieldSchemas = - new ResourceFieldSchema[] {convert(listType.elementType())}; - - if (listType.elementType().isStructType()) { - result.setFields(elementFieldSchemas); - } else { - // Wrap non-struct types in tuples - ResourceSchema elementSchema = new ResourceSchema(); - elementSchema.setFields(elementFieldSchemas); - - ResourceFieldSchema tupleSchema = new ResourceFieldSchema(); - tupleSchema.setType(DataType.TUPLE); - tupleSchema.setSchema(elementSchema); - - result.setFields(new ResourceFieldSchema[] {tupleSchema}); - } - - return result; - case MAP: - Types.MapType mapType = type.asMapType(); - - if (mapType.keyType().typeId() != Type.TypeID.STRING) { - throw new FrontendException("Unsupported map key type: " + mapType.keyType()); - } - result.setFields(new ResourceFieldSchema[] {convert(mapType.valueType())}); - - return result; - default: - throw new FrontendException("Unsupported complex type: " + type); - } - } - - public static Schema project(Schema schema, List requiredFields) { - List columns = Lists.newArrayList(); - - for (String column : requiredFields) { - columns.add(schema.findField(column)); - } - - return new Schema(columns); - } -} diff --git a/pig/src/test/java/org/apache/iceberg/pig/SchemaUtilTest.java b/pig/src/test/java/org/apache/iceberg/pig/SchemaUtilTest.java deleted file mode 100644 index f7136045a1f5..000000000000 --- a/pig/src/test/java/org/apache/iceberg/pig/SchemaUtilTest.java +++ /dev/null @@ -1,287 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.pig; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.io.IOException; -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.Types.BinaryType; -import org.apache.iceberg.types.Types.BooleanType; -import org.apache.iceberg.types.Types.DecimalType; -import org.apache.iceberg.types.Types.DoubleType; -import org.apache.iceberg.types.Types.FloatType; -import org.apache.iceberg.types.Types.IntegerType; -import org.apache.iceberg.types.Types.ListType; -import org.apache.iceberg.types.Types.LongType; -import org.apache.iceberg.types.Types.MapType; -import org.apache.iceberg.types.Types.StringType; -import org.apache.iceberg.types.Types.StructType; -import org.apache.pig.ResourceSchema; -import org.apache.pig.impl.logicalLayer.FrontendException; -import org.junit.jupiter.api.Test; - -public class SchemaUtilTest { - - @Test - public void testPrimitive() throws IOException { - Schema icebergSchema = - new Schema( - optional(1, "b", BooleanType.get()), - optional(2, "i", IntegerType.get()), - optional(3, "l", LongType.get()), - optional(4, "f", FloatType.get()), - optional(5, "d", DoubleType.get()), - optional(6, "dec", DecimalType.of(0, 2)), - optional(7, "s", StringType.get()), - optional(8, "bi", BinaryType.get())); - - ResourceSchema pigSchema = SchemaUtil.convert(icebergSchema); - assertThat(pigSchema.toString()) - .isEqualTo( - "b:boolean,i:int,l:long,f:float,d:double,dec:bigdecimal,s:chararray,bi:bytearray"); - } - - @Test - public void testComplex() throws IOException { - convertToPigSchema( - new Schema( - optional(1, "bag", ListType.ofOptional(2, BooleanType.get())), - optional(3, "map", MapType.ofOptional(4, 5, StringType.get(), DoubleType.get())), - optional( - 6, - "tuple", - StructType.of( - optional(7, "i", IntegerType.get()), optional(8, "f", FloatType.get())))), - "bag:{(boolean)},map:[double],tuple:(i:int,f:float)", - null); - } - - @Test - public void invalidMap() { - assertThatThrownBy( - () -> - convertToPigSchema( - new Schema( - optional( - 1, - "invalid", - MapType.ofOptional(2, 3, IntegerType.get(), DoubleType.get()))), - "", - "")) - .isInstanceOf(FrontendException.class) - .hasMessageContaining("Unsupported map key type: int"); - } - - @Test - public void nestedMaps() throws IOException { - convertToPigSchema( - new Schema( - optional( - 1, - "nested", - MapType.ofOptional( - 2, - 3, - StringType.get(), - MapType.ofOptional( - 4, - 5, - StringType.get(), - MapType.ofOptional(6, 7, StringType.get(), DecimalType.of(10, 2)))))), - "nested:[[[bigdecimal]]]", - ""); - } - - @Test - public void nestedBags() throws IOException { - convertToPigSchema( - new Schema( - optional( - 1, - "nested", - ListType.ofOptional( - 2, ListType.ofOptional(3, ListType.ofOptional(4, DoubleType.get()))))), - "nested:{({({(double)})})}", - ""); - } - - @Test - public void nestedTuples() throws IOException { - convertToPigSchema( - new Schema( - optional( - 1, - "first", - StructType.of( - optional( - 2, - "second", - StructType.of( - optional( - 3, - "third", - StructType.of(optional(4, "val", StringType.get())))))))), - "first:(second:(third:(val:chararray)))", - ""); - } - - @Test - public void complexNested() throws IOException { - convertToPigSchema( - new Schema( - optional( - 1, - "t", - StructType.of( - optional( - 2, - "b", - ListType.ofOptional( - 3, - StructType.of( - optional(4, "i", IntegerType.get()), - optional(5, "s", StringType.get())))))), - optional( - 6, - "m1", - MapType.ofOptional( - 7, - 8, - StringType.get(), - StructType.of( - optional(9, "b", ListType.ofOptional(10, BinaryType.get())), - optional( - 11, - "m2", - MapType.ofOptional(12, 13, StringType.get(), IntegerType.get()))))), - optional( - 14, - "b1", - ListType.ofOptional( - 15, - MapType.ofOptional( - 16, 17, StringType.get(), ListType.ofOptional(18, FloatType.get()))))), - "t:(b:{(i:int,s:chararray)}),m1:[(b:{(bytearray)},m2:[int])],b1:{([{(float)}])}", - ""); - } - - @Test - public void mapConversions() throws IOException { - // consistent behavior for maps conversions. The below test case, correctly does not specify map - // key types - convertToPigSchema( - new Schema( - required( - 1, - "a", - MapType.ofRequired( - 2, - 3, - StringType.get(), - ListType.ofRequired( - 4, - StructType.of( - required(5, "b", LongType.get()), - required(6, "c", StringType.get())))))), - "a:[{(b:long,c:chararray)}]", - "We do not specify the map key type here"); - // struct>> -> (a:[[double]]) - // As per https://pig.apache.org/docs/latest/basic.html#map-schema. It seems that - // we only need to specify value type as keys are always of type chararray - convertToPigSchema( - new Schema( - StructType.of( - required( - 1, - "a", - MapType.ofRequired( - 2, - 3, - StringType.get(), - MapType.ofRequired(4, 5, StringType.get(), DoubleType.get())))) - .fields()), - "a:[[double]]", - "A map key type does not need to be specified"); - } - - @Test - public void testTupleInMap() throws IOException { - Schema icebergSchema = - new Schema( - optional( - 1, - "nested_list", - MapType.ofOptional( - 2, - 3, - StringType.get(), - ListType.ofOptional( - 4, - StructType.of( - required(5, "id", LongType.get()), - optional(6, "data", StringType.get())))))); - - ResourceSchema pigSchema = SchemaUtil.convert(icebergSchema); - // The output should contain a nested struct within a list within a map, I think. - assertThat(pigSchema.toString()).isEqualTo("nested_list:[{(id:long,data:chararray)}]"); - } - - @Test - public void testLongInBag() throws IOException { - Schema icebergSchema = - new Schema( - optional( - 1, - "nested_list", - MapType.ofOptional( - 2, 3, StringType.get(), ListType.ofRequired(5, LongType.get())))); - SchemaUtil.convert(icebergSchema); - } - - @Test - public void doubleWrappingTuples() throws IOException { - // struct>> -> (a:{(b:chararray)}) - convertToPigSchema( - new Schema( - StructType.of( - required( - 1, - "a", - ListType.ofRequired(2, StructType.of(required(3, "b", StringType.get()))))) - .fields()), - "a:{(b:chararray)}", - "A tuple inside a bag should not be double wrapped"); - // struct> -> "(a:{(boolean)}) - convertToPigSchema( - new Schema( - StructType.of(required(1, "a", ListType.ofRequired(2, BooleanType.get()))).fields()), - "a:{(boolean)}", - "boolean (or anything non-tuple) element inside a bag should be wrapped inside a tuple"); - } - - private static void convertToPigSchema( - Schema icebergSchema, String expectedPigSchema, String assertMessage) throws IOException { - ResourceSchema pigSchema = SchemaUtil.convert(icebergSchema); - assertThat(pigSchema.toString()).as(assertMessage).isEqualTo(expectedPigSchema); - } -} diff --git a/settings.gradle b/settings.gradle index 56a68c384c5f..103741389a26 100644 --- a/settings.gradle +++ b/settings.gradle @@ -33,7 +33,6 @@ include 'arrow' include 'parquet' include 'bundled-guava' include 'spark' -include 'pig' include 'hive-metastore' include 'nessie' include 'gcp' @@ -58,7 +57,6 @@ project(':arrow').name = 'iceberg-arrow' project(':parquet').name = 'iceberg-parquet' project(':bundled-guava').name = 'iceberg-bundled-guava' project(':spark').name = 'iceberg-spark' -project(':pig').name = 'iceberg-pig' project(':hive-metastore').name = 'iceberg-hive-metastore' project(':nessie').name = 'iceberg-nessie' project(':gcp').name = 'iceberg-gcp' diff --git a/site/docs/contribute.md b/site/docs/contribute.md index 94b4679da78c..a12936a7bc49 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -119,7 +119,6 @@ This project Iceberg also has modules for adding Iceberg support to processing e * `iceberg-spark` is an implementation of Spark's Datasource V2 API for Iceberg with submodules for each spark versions (use runtime jars for a shaded version) * `iceberg-flink` contains classes for integrating with Apache Flink (use iceberg-flink-runtime for a shaded version) * `iceberg-mr` contains an InputFormat and other classes for integrating with Apache Hive -* `iceberg-pig` is an implementation of Pig's LoadFunc API for Iceberg ## Setting up IDE and Code Style From e06b069529be3d3d389b156646e751de3753feb0 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 13 Nov 2024 16:42:06 +0100 Subject: [PATCH 135/313] Core, Flink, Spark: Test DVs with format-version=3 (#11485) --- .../apache/iceberg/BaseContentScanTask.java | 3 +- .../org/apache/iceberg/BaseFileScanTask.java | 2 + .../iceberg/TestMetadataTableFilters.java | 68 ++--- .../iceberg/TestMetadataTableScans.java | 2 +- .../org/apache/iceberg/data/FileHelpers.java | 98 ++++-- .../actions/TestRewriteDataFilesAction.java | 27 +- .../TestDeleteReachableFilesAction.java | 54 +++- .../actions/TestExpireSnapshotsAction.java | 102 ++++--- .../TestRemoveDanglingDeleteAction.java | 116 +++++--- .../actions/TestRemoveOrphanFilesAction.java | 147 ++++----- .../actions/TestRemoveOrphanFilesAction3.java | 71 +++-- .../actions/TestRewriteDataFilesAction.java | 279 ++++++++++++------ .../actions/TestRewriteManifestsAction.java | 28 +- 13 files changed, 624 insertions(+), 373 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java b/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java index 8d38a48309c6..960c04cc0f37 100644 --- a/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java @@ -23,6 +23,7 @@ import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.ScanTaskUtil; abstract class BaseContentScanTask, F extends ContentFile> implements ContentScanTask, SplittableScanTask { @@ -82,7 +83,7 @@ public long start() { @Override public long length() { - return file.fileSizeInBytes(); + return ScanTaskUtil.contentSizeInBytes(file); } @Override diff --git a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java index aa37f40be7c0..2cc406444552 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java @@ -176,6 +176,8 @@ public boolean canMerge(ScanTask other) { @Override public SplitScanTask merge(ScanTask other) { SplitScanTask that = (SplitScanTask) other; + // don't use deletesSizeBytes() here so that deletesSizeBytes is only calculated once after + // merging rather than for each task before merging return new SplitScanTask(offset, len + that.length(), fileScanTask, deletesSizeBytes); } diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index f8c34019875f..7c5a860db15f 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -52,18 +52,26 @@ protected static List parameters() { return Arrays.asList( new Object[] {1, MetadataTableType.DATA_FILES}, new Object[] {2, MetadataTableType.DATA_FILES}, + new Object[] {3, MetadataTableType.DATA_FILES}, new Object[] {2, MetadataTableType.DELETE_FILES}, + new Object[] {3, MetadataTableType.DELETE_FILES}, new Object[] {1, MetadataTableType.FILES}, new Object[] {2, MetadataTableType.FILES}, + new Object[] {3, MetadataTableType.FILES}, new Object[] {1, MetadataTableType.ALL_DATA_FILES}, new Object[] {2, MetadataTableType.ALL_DATA_FILES}, + new Object[] {3, MetadataTableType.ALL_DATA_FILES}, new Object[] {2, MetadataTableType.ALL_DELETE_FILES}, + new Object[] {3, MetadataTableType.ALL_DELETE_FILES}, new Object[] {1, MetadataTableType.ALL_FILES}, new Object[] {2, MetadataTableType.ALL_FILES}, + new Object[] {3, MetadataTableType.ALL_FILES}, new Object[] {1, MetadataTableType.ENTRIES}, new Object[] {2, MetadataTableType.ENTRIES}, + new Object[] {3, MetadataTableType.ENTRIES}, new Object[] {1, MetadataTableType.ALL_ENTRIES}, - new Object[] {2, MetadataTableType.ALL_ENTRIES}); + new Object[] {2, MetadataTableType.ALL_ENTRIES}, + new Object[] {3, MetadataTableType.ALL_ENTRIES}); } @BeforeEach @@ -76,9 +84,9 @@ public void setupTable() throws Exception { table.newFastAppend().appendFile(FILE_D).commit(); table.newFastAppend().appendFile(FILE_B).commit(); - if (formatVersion == 2) { - table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); - table.newRowDelta().addDeletes(FILE_B_DELETES).commit(); + if (formatVersion >= 2) { + table.newRowDelta().addDeletes(fileADeletes()).commit(); + table.newRowDelta().addDeletes(fileBDeletes()).commit(); table.newRowDelta().addDeletes(FILE_C2_DELETES).commit(); table.newRowDelta().addDeletes(FILE_D2_DELETES).commit(); } @@ -366,7 +374,7 @@ public void testPartitionSpecEvolutionRemovalV1() { @TestTemplate public void testPartitionSpecEvolutionRemovalV2() { - assumeThat(formatVersion).isEqualTo(2); + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); // Change spec and add two data and delete files each table.updateSpec().removeField(Expressions.bucket("data", 16)).addField("id").commit(); @@ -388,27 +396,13 @@ public void testPartitionSpecEvolutionRemovalV2() { .withPartitionPath("id=11") .build(); - DeleteFile delete10 = - FileMetadata.deleteFileBuilder(newSpec) - .ofPositionDeletes() - .withPath("/path/to/data-10-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("id=10") - .withRecordCount(1) - .build(); - DeleteFile delete11 = - FileMetadata.deleteFileBuilder(newSpec) - .ofPositionDeletes() - .withPath("/path/to/data-11-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("id=11") - .withRecordCount(1) - .build(); + DeleteFile delete10 = posDelete(table, data10); + DeleteFile delete11 = posDelete(table, data11); table.newFastAppend().appendFile(data10).commit(); table.newFastAppend().appendFile(data11).commit(); - if (formatVersion == 2) { + if (formatVersion >= 2) { table.newRowDelta().addDeletes(delete10).commit(); table.newRowDelta().addDeletes(delete11).commit(); } @@ -447,6 +441,12 @@ public void testPartitionSpecEvolutionRemovalV2() { assertThat(tasks).hasSize(expectedScanTaskCount(3)); } + private DeleteFile posDelete(Table table, DataFile dataFile) { + return formatVersion >= 3 + ? FileGenerationUtil.generateDV(table, dataFile) + : FileGenerationUtil.generatePositionDeleteFile(table, dataFile); + } + @TestTemplate public void testPartitionSpecEvolutionAdditiveV1() { assumeThat(formatVersion).isEqualTo(1); @@ -514,8 +514,8 @@ public void testPartitionSpecEvolutionAdditiveV1() { } @TestTemplate - public void testPartitionSpecEvolutionAdditiveV2() { - assumeThat(formatVersion).isEqualTo(2); + public void testPartitionSpecEvolutionAdditiveV2AndAbove() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); // Change spec and add two data and delete files each table.updateSpec().addField("id").commit(); @@ -537,27 +537,13 @@ public void testPartitionSpecEvolutionAdditiveV2() { .withPartitionPath("data_bucket=1/id=11") .build(); - DeleteFile delete10 = - FileMetadata.deleteFileBuilder(newSpec) - .ofPositionDeletes() - .withPath("/path/to/data-10-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=0/id=10") - .withRecordCount(1) - .build(); - DeleteFile delete11 = - FileMetadata.deleteFileBuilder(newSpec) - .ofPositionDeletes() - .withPath("/path/to/data-11-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=1/id=11") - .withRecordCount(1) - .build(); + DeleteFile delete10 = posDelete(table, data10); + DeleteFile delete11 = posDelete(table, data11); table.newFastAppend().appendFile(data10).commit(); table.newFastAppend().appendFile(data11).commit(); - if (formatVersion == 2) { + if (formatVersion >= 2) { table.newRowDelta().addDeletes(delete10).commit(); table.newRowDelta().addDeletes(delete11).commit(); } diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index f811dac02043..a31e02144167 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -1733,7 +1733,7 @@ public void testFilesTableEstimateSize() throws Exception { assertEstimatedRowCount(new AllDataFilesTable(table), 4); assertEstimatedRowCount(new AllFilesTable(table), 4); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertEstimatedRowCount(new DeleteFilesTable(table), 4); assertEstimatedRowCount(new AllDeleteFilesTable(table), 4); } diff --git a/data/src/test/java/org/apache/iceberg/data/FileHelpers.java b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java index 62df1634aa3f..181ca18138a1 100644 --- a/data/src/test/java/org/apache/iceberg/data/FileHelpers.java +++ b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java @@ -23,9 +23,12 @@ import java.util.List; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteWriter; @@ -35,6 +38,8 @@ import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.Pair; @@ -47,21 +52,53 @@ public static Pair writeDeleteFile( return writeDeleteFile(table, out, null, deletes); } + public static Pair writeDeleteFile( + Table table, OutputFile out, List> deletes, int formatVersion) + throws IOException { + return writeDeleteFile(table, out, null, deletes, formatVersion); + } + public static Pair writeDeleteFile( Table table, OutputFile out, StructLike partition, List> deletes) throws IOException { - FileWriterFactory factory = GenericFileWriterFactory.builderFor(table).build(); + return writeDeleteFile(table, out, partition, deletes, 2); + } - PositionDeleteWriter writer = - factory.newPositionDeleteWriter(encrypt(out), table.spec(), partition); - PositionDelete posDelete = PositionDelete.create(); - try (Closeable toClose = writer) { - for (Pair delete : deletes) { - writer.write(posDelete.set(delete.first(), delete.second(), null)); + public static Pair writeDeleteFile( + Table table, + OutputFile out, + StructLike partition, + List> deletes, + int formatVersion) + throws IOException { + if (formatVersion >= 3) { + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PUFFIN).build(); + DVFileWriter writer = new BaseDVFileWriter(fileFactory, p -> null); + try (DVFileWriter closeableWriter = writer) { + for (Pair delete : deletes) { + closeableWriter.delete( + delete.first().toString(), delete.second(), table.spec(), partition); + } + } + + return Pair.of( + Iterables.getOnlyElement(writer.result().deleteFiles()), + writer.result().referencedDataFiles()); + } else { + FileWriterFactory factory = GenericFileWriterFactory.builderFor(table).build(); + + PositionDeleteWriter writer = + factory.newPositionDeleteWriter(encrypt(out), table.spec(), partition); + PositionDelete posDelete = PositionDelete.create(); + try (Closeable toClose = writer) { + for (Pair delete : deletes) { + writer.write(posDelete.set(delete.first(), delete.second(), null)); + } } - } - return Pair.of(writer.toDeleteFile(), writer.referencedDataFiles()); + return Pair.of(writer.toDeleteFile(), writer.referencedDataFiles()); + } } public static DeleteFile writeDeleteFile( @@ -121,18 +158,43 @@ public static DataFile writeDataFile( public static DeleteFile writePosDeleteFile( Table table, OutputFile out, StructLike partition, List> deletes) throws IOException { - FileWriterFactory factory = - GenericFileWriterFactory.builderFor(table).positionDeleteRowSchema(table.schema()).build(); + return writePosDeleteFile(table, out, partition, deletes, 2); + } - PositionDeleteWriter writer = - factory.newPositionDeleteWriter(encrypt(out), table.spec(), partition); - try (Closeable toClose = writer) { - for (PositionDelete delete : deletes) { - writer.write(delete); + public static DeleteFile writePosDeleteFile( + Table table, + OutputFile out, + StructLike partition, + List> deletes, + int formatVersion) + throws IOException { + if (formatVersion >= 3) { + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PUFFIN).build(); + DVFileWriter writer = new BaseDVFileWriter(fileFactory, p -> null); + try (DVFileWriter closeableWriter = writer) { + for (PositionDelete delete : deletes) { + closeableWriter.delete(delete.path().toString(), delete.pos(), table.spec(), partition); + } } - } - return writer.toDeleteFile(); + return Iterables.getOnlyElement(writer.result().deleteFiles()); + } else { + FileWriterFactory factory = + GenericFileWriterFactory.builderFor(table) + .positionDeleteRowSchema(table.schema()) + .build(); + + PositionDeleteWriter writer = + factory.newPositionDeleteWriter(encrypt(out), table.spec(), partition); + try (Closeable toClose = writer) { + for (PositionDelete delete : deletes) { + writer.write(delete); + } + } + + return writer.toDeleteFile(); + } } private static EncryptedOutputFile encrypt(OutputFile out) { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java index 6b8399f666d4..085e7e48204c 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -25,6 +25,7 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Set; @@ -46,6 +47,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.actions.RewriteDataFilesActionResult; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -77,6 +79,9 @@ public class TestRewriteDataFilesAction extends CatalogTestBase { @Parameter(index = 2) private FileFormat format; + @Parameter(index = 3) + private int formatVersion; + private Table icebergTableUnPartitioned; private Table icebergTablePartitioned; private Table icebergTableWithPk; @@ -87,15 +92,17 @@ protected TableEnvironment getTableEnv() { return super.getTableEnv(); } - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}") + @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, formatVersion={3}") public static List parameters() { List parameters = Lists.newArrayList(); for (FileFormat format : new FileFormat[] {FileFormat.AVRO, FileFormat.ORC, FileFormat.PARQUET}) { for (Object[] catalogParams : CatalogTestBase.parameters()) { - String catalogName = (String) catalogParams[0]; - Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format}); + for (int version : Arrays.asList(2, 3)) { + String catalogName = (String) catalogParams[0]; + Namespace baseNamespace = (Namespace) catalogParams[1]; + parameters.add(new Object[] {catalogName, baseNamespace, format, version}); + } } } return parameters; @@ -111,21 +118,21 @@ public void before() { sql("USE CATALOG %s", catalogName); sql("USE %s", DATABASE); sql( - "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", - TABLE_NAME_UNPARTITIONED, format.name()); + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s', '%s'='%s')", + TABLE_NAME_UNPARTITIONED, format.name(), TableProperties.FORMAT_VERSION, formatVersion); icebergTableUnPartitioned = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_UNPARTITIONED)); sql( "CREATE TABLE %s (id int, data varchar,spec varchar) " - + " PARTITIONED BY (data,spec) with ('write.format.default'='%s')", - TABLE_NAME_PARTITIONED, format.name()); + + " PARTITIONED BY (data,spec) with ('write.format.default'='%s', '%s'='%s')", + TABLE_NAME_PARTITIONED, format.name(), TableProperties.FORMAT_VERSION, formatVersion); icebergTablePartitioned = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_PARTITIONED)); sql( - "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) with ('write.format.default'='%s', 'format-version'='2')", - TABLE_NAME_WITH_PK, format.name()); + "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) with ('write.format.default'='%s', '%s'='%s')", + TABLE_NAME_WITH_PK, format.name(), TableProperties.FORMAT_VERSION, formatVersion); icebergTableWithPk = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index bfa09552396a..ad93b80baf81 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -23,6 +23,8 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; +import java.util.Arrays; +import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; @@ -32,8 +34,12 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; import org.apache.iceberg.FileMetadata; import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -47,14 +53,15 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +@ExtendWith(ParameterizedTestExtension.class) public class TestDeleteReachableFilesAction extends TestBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); private static final Schema SCHEMA = @@ -112,13 +119,24 @@ public class TestDeleteReachableFilesAction extends TestBase { .build(); @TempDir private File tableDir; + @Parameter private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(2, 3); + } private Table table; @BeforeEach public void setupTableLocation() throws Exception { String tableLocation = tableDir.toURI().toString(); - this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); + this.table = + TABLES.create( + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + tableLocation); spark.conf().set("spark.sql.shuffle.partitions", SHUFFLE_PARTITIONS); } @@ -155,7 +173,7 @@ private void checkRemoveFilesResults( .isEqualTo(expectedOtherFilesDeleted); } - @Test + @TestTemplate public void dataFilesCleanupWithParallelTasks() { table.newFastAppend().appendFile(FILE_A).commit(); @@ -206,7 +224,7 @@ public void dataFilesCleanupWithParallelTasks() { checkRemoveFilesResults(4L, 0, 0, 6L, 4L, 6, result); } - @Test + @TestTemplate public void testWithExpiringDanglingStageCommit() { table.location(); // `A` commit @@ -224,7 +242,7 @@ public void testWithExpiringDanglingStageCommit() { checkRemoveFilesResults(3L, 0, 0, 3L, 3L, 5, result); } - @Test + @TestTemplate public void testRemoveFileActionOnEmptyTable() { DeleteReachableFiles.Result result = sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()).execute(); @@ -232,7 +250,7 @@ public void testRemoveFileActionOnEmptyTable() { checkRemoveFilesResults(0, 0, 0, 0, 0, 2, result); } - @Test + @TestTemplate public void testRemoveFilesActionWithReducedVersionsTable() { table.updateProperties().set(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, "2").commit(); table.newAppend().appendFile(FILE_A).commit(); @@ -252,7 +270,7 @@ public void testRemoveFilesActionWithReducedVersionsTable() { checkRemoveFilesResults(4, 0, 0, 5, 5, 8, result); } - @Test + @TestTemplate public void testRemoveFilesAction() { table.newAppend().appendFile(FILE_A).commit(); @@ -263,20 +281,20 @@ public void testRemoveFilesAction() { checkRemoveFilesResults(2, 0, 0, 2, 2, 4, baseRemoveFilesSparkAction.execute()); } - @Test + @TestTemplate public void testPositionDeleteFiles() { table.newAppend().appendFile(FILE_A).commit(); table.newAppend().appendFile(FILE_B).commit(); - table.newRowDelta().addDeletes(FILE_A_POS_DELETES).commit(); + table.newRowDelta().addDeletes(fileADeletes()).commit(); DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()); checkRemoveFilesResults(2, 1, 0, 3, 3, 5, baseRemoveFilesSparkAction.execute()); } - @Test + @TestTemplate public void testEqualityDeleteFiles() { table.newAppend().appendFile(FILE_A).commit(); @@ -289,7 +307,7 @@ public void testEqualityDeleteFiles() { checkRemoveFilesResults(2, 0, 1, 3, 3, 5, baseRemoveFilesSparkAction.execute()); } - @Test + @TestTemplate public void testRemoveFilesActionWithDefaultIO() { table.newAppend().appendFile(FILE_A).commit(); @@ -302,7 +320,7 @@ public void testRemoveFilesActionWithDefaultIO() { checkRemoveFilesResults(2, 0, 0, 2, 2, 4, baseRemoveFilesSparkAction.execute()); } - @Test + @TestTemplate public void testUseLocalIterator() { table.newFastAppend().appendFile(FILE_A).commit(); @@ -333,7 +351,7 @@ public void testUseLocalIterator() { }); } - @Test + @TestTemplate public void testIgnoreMetadataFilesNotFound() { table.updateProperties().set(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, "1").commit(); @@ -354,7 +372,7 @@ public void testIgnoreMetadataFilesNotFound() { checkRemoveFilesResults(1, 0, 0, 1, 1, 4, res); } - @Test + @TestTemplate public void testEmptyIOThrowsException() { DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions().deleteReachableFiles(metadataLocation(table)).io(null); @@ -364,7 +382,7 @@ public void testEmptyIOThrowsException() { .hasMessage("File IO cannot be null"); } - @Test + @TestTemplate public void testRemoveFilesActionWhenGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); @@ -381,4 +399,8 @@ private String metadataLocation(Table tbl) { private ActionsProvider sparkActions() { return SparkActions.get(); } + + private DeleteFile fileADeletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_A) : FILE_A_POS_DELETES; + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 5909dec51c05..661df99ef3e6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -21,10 +21,12 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; import java.nio.file.Path; +import java.util.Arrays; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -36,8 +38,12 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; import org.apache.iceberg.FileMetadata; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReachableFileUtil; import org.apache.iceberg.Schema; @@ -53,16 +59,17 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +@ExtendWith(ParameterizedTestExtension.class) public class TestExpireSnapshotsAction extends TestBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); private static final Schema SCHEMA = @@ -120,6 +127,12 @@ public class TestExpireSnapshotsAction extends TestBase { .build(); @TempDir private Path temp; + @Parameter private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(2, 3); + } @TempDir private File tableDir; private String tableLocation; @@ -128,7 +141,12 @@ public class TestExpireSnapshotsAction extends TestBase { @BeforeEach public void setupTableLocation() throws Exception { this.tableLocation = tableDir.toURI().toString(); - this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); + this.table = + TABLES.create( + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + tableLocation); spark.conf().set("spark.sql.shuffle.partitions", SHUFFLE_PARTITIONS); } @@ -144,6 +162,10 @@ private Long rightAfterSnapshot(long snapshotId) { return end; } + private DeleteFile fileADeletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_A) : FILE_A_POS_DELETES; + } + private void checkExpirationResults( long expectedDatafiles, long expectedPosDeleteFiles, @@ -173,7 +195,7 @@ private void checkExpirationResults( .isEqualTo(expectedManifestListsDeleted); } - @Test + @TestTemplate public void testFilesCleaned() throws Exception { table.newFastAppend().appendFile(FILE_A).commit(); @@ -191,7 +213,7 @@ public void testFilesCleaned() throws Exception { checkExpirationResults(1L, 0L, 0L, 1L, 2L, results); } - @Test + @TestTemplate public void dataFilesCleanupWithParallelTasks() throws IOException { table.newFastAppend().appendFile(FILE_A).commit(); @@ -245,7 +267,7 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { checkExpirationResults(2L, 0L, 0L, 3L, 3L, result); } - @Test + @TestTemplate public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception { table.newFastAppend().appendFile(FILE_A).commit(); @@ -253,7 +275,7 @@ public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception { checkExpirationResults(0L, 0L, 0L, 0L, 0L, results); } - @Test + @TestTemplate public void testCleanupRepeatedOverwrites() throws Exception { table.newFastAppend().appendFile(FILE_A).commit(); @@ -269,7 +291,7 @@ public void testCleanupRepeatedOverwrites() throws Exception { checkExpirationResults(1L, 0L, 0L, 39L, 20L, results); } - @Test + @TestTemplate public void testRetainLastWithExpireOlderThan() { table .newAppend() @@ -300,7 +322,7 @@ public void testRetainLastWithExpireOlderThan() { assertThat(table.snapshot(firstSnapshotId)).as("First snapshot should not present.").isNull(); } - @Test + @TestTemplate public void testExpireTwoSnapshotsById() throws Exception { table .newAppend() @@ -335,7 +357,7 @@ public void testExpireTwoSnapshotsById() throws Exception { checkExpirationResults(0L, 0L, 0L, 0L, 2L, result); } - @Test + @TestTemplate public void testRetainLastWithExpireById() { table .newAppend() @@ -366,7 +388,7 @@ public void testRetainLastWithExpireById() { checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); } - @Test + @TestTemplate public void testRetainLastWithTooFewSnapshots() { table .newAppend() @@ -393,7 +415,7 @@ public void testRetainLastWithTooFewSnapshots() { checkExpirationResults(0L, 0L, 0L, 0L, 0L, result); } - @Test + @TestTemplate public void testRetainLastKeepsExpiringSnapshot() { table .newAppend() @@ -432,7 +454,7 @@ public void testRetainLastKeepsExpiringSnapshot() { checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); } - @Test + @TestTemplate public void testExpireSnapshotsWithDisabledGarbageCollection() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); @@ -444,7 +466,7 @@ public void testExpireSnapshotsWithDisabledGarbageCollection() { "Cannot expire snapshots: GC is disabled (deleting files may corrupt other tables)"); } - @Test + @TestTemplate public void testExpireOlderThanMultipleCalls() { table .newAppend() @@ -480,7 +502,7 @@ public void testExpireOlderThanMultipleCalls() { checkExpirationResults(0L, 0L, 0L, 0L, 2L, result); } - @Test + @TestTemplate public void testRetainLastMultipleCalls() { table .newAppend() @@ -517,14 +539,14 @@ public void testRetainLastMultipleCalls() { checkExpirationResults(0L, 0L, 0L, 0L, 2L, result); } - @Test + @TestTemplate public void testRetainZeroSnapshots() { assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table).retainLast(0).execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Number of snapshots to retain must be at least 1, cannot be: 0"); } - @Test + @TestTemplate public void testScanExpiredManifestInValidSnapshotAppend() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -547,7 +569,7 @@ public void testScanExpiredManifestInValidSnapshotAppend() { checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } - @Test + @TestTemplate public void testScanExpiredManifestInValidSnapshotFastAppend() { table .updateProperties() @@ -580,7 +602,7 @@ public void testScanExpiredManifestInValidSnapshotFastAppend() { * Test on table below, and expiring the staged commit `B` using `expireOlderThan` API. Table: A - * C ` B (staged) */ - @Test + @TestTemplate public void testWithExpiringDanglingStageCommit() { // `A` commit table.newAppend().appendFile(FILE_A).commit(); @@ -641,7 +663,7 @@ public void testWithExpiringDanglingStageCommit() { * Expire cherry-pick the commit as shown below, when `B` is in table's current state Table: A - B * - C <--current snapshot `- D (source=B) */ - @Test + @TestTemplate public void testWithCherryPickTableSnapshot() { // `A` commit table.newAppend().appendFile(FILE_A).commit(); @@ -696,7 +718,7 @@ public void testWithCherryPickTableSnapshot() { * Test on table below, and expiring `B` which is not in current table state. 1) Expire `B` 2) All * commit Table: A - C - D (B) ` B (staged) */ - @Test + @TestTemplate public void testWithExpiringStagedThenCherrypick() { // `A` commit table.newAppend().appendFile(FILE_A).commit(); @@ -760,7 +782,7 @@ public void testWithExpiringStagedThenCherrypick() { checkExpirationResults(0L, 0L, 0L, 0L, 2L, secondResult); } - @Test + @TestTemplate public void testExpireOlderThan() { table.newAppend().appendFile(FILE_A).commit(); @@ -796,7 +818,7 @@ public void testExpireOlderThan() { checkExpirationResults(0, 0, 0, 0, 1, result); } - @Test + @TestTemplate public void testExpireOlderThanWithDelete() { table.newAppend().appendFile(FILE_A).commit(); @@ -858,7 +880,7 @@ public void testExpireOlderThanWithDelete() { checkExpirationResults(1, 0, 0, 2, 2, result); } - @Test + @TestTemplate public void testExpireOlderThanWithDeleteInMergedManifests() { // merge every commit table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0").commit(); @@ -924,7 +946,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { checkExpirationResults(1, 0, 0, 1, 2, result); } - @Test + @TestTemplate public void testExpireOlderThanWithRollback() { // merge every commit table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0").commit(); @@ -982,7 +1004,7 @@ public void testExpireOlderThanWithRollback() { checkExpirationResults(0, 0, 0, 1, 1, result); } - @Test + @TestTemplate public void testExpireOlderThanWithRollbackAndMergedManifests() { table.newAppend().appendFile(FILE_A).commit(); @@ -1037,20 +1059,18 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { checkExpirationResults(1, 0, 0, 1, 1, result); } - @Test + @TestTemplate public void testExpireOlderThanWithDeleteFile() { - table - .updateProperties() - .set(TableProperties.FORMAT_VERSION, "2") - .set(TableProperties.MANIFEST_MERGE_ENABLED, "false") - .commit(); + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + table.updateProperties().set(TableProperties.MANIFEST_MERGE_ENABLED, "false").commit(); // Add Data File table.newAppend().appendFile(FILE_A).commit(); Snapshot firstSnapshot = table.currentSnapshot(); // Add POS Delete - table.newRowDelta().addDeletes(FILE_A_POS_DELETES).commit(); + DeleteFile fileADeletes = fileADeletes(); + table.newRowDelta().addDeletes(fileADeletes).commit(); Snapshot secondSnapshot = table.currentSnapshot(); // Add EQ Delete @@ -1081,7 +1101,7 @@ public void testExpireOlderThanWithDeleteFile() { thirdSnapshot.manifestListLocation(), fourthSnapshot.manifestListLocation(), FILE_A.path().toString(), - FILE_A_POS_DELETES.path().toString(), + fileADeletes.path().toString(), FILE_A_EQ_DELETES.path().toString()); expectedDeletes.addAll( @@ -1103,7 +1123,7 @@ public void testExpireOlderThanWithDeleteFile() { checkExpirationResults(1, 1, 1, 6, 4, result); } - @Test + @TestTemplate public void testExpireOnEmptyTable() { Set deletedFiles = Sets.newHashSet(); @@ -1118,7 +1138,7 @@ public void testExpireOnEmptyTable() { checkExpirationResults(0, 0, 0, 0, 0, result); } - @Test + @TestTemplate public void testExpireAction() { table.newAppend().appendFile(FILE_A).commit(); @@ -1167,7 +1187,7 @@ public void testExpireAction() { .isEqualTo(pendingDeletes.count()); } - @Test + @TestTemplate public void testUseLocalIterator() { table.newFastAppend().appendFile(FILE_A).commit(); @@ -1201,7 +1221,7 @@ public void testUseLocalIterator() { }); } - @Test + @TestTemplate public void testExpireAfterExecute() { table .newAppend() @@ -1236,12 +1256,12 @@ public void testExpireAfterExecute() { assertThat(untypedExpiredFiles).as("Expired results must match").hasSize(1); } - @Test + @TestTemplate public void testExpireFileDeletionMostExpired() { textExpireAllCheckFilesDeleted(5, 2); } - @Test + @TestTemplate public void testExpireFileDeletionMostRetained() { textExpireAllCheckFilesDeleted(2, 5); } @@ -1303,7 +1323,7 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe .isEqualTo(expectedDeletes); } - @Test + @TestTemplate public void testExpireSomeCheckFilesDeleted() { table.newAppend().appendFile(FILE_A).commit(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java index 3b4dce73fee5..e58966cfea3f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.File; +import java.util.Arrays; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -30,7 +31,11 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -44,10 +49,12 @@ import org.apache.spark.sql.Encoders; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; import scala.Tuple2; +@ExtendWith(ParameterizedTestExtension.class) public class TestRemoveDanglingDeleteAction extends TestBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); @@ -202,6 +209,12 @@ public class TestRemoveDanglingDeleteAction extends TestBase { .build(); @TempDir private File tableDir; + @Parameter private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(2, 3); + } private String tableLocation = null; private Table table; @@ -219,7 +232,10 @@ public void after() { private void setupPartitionedTable() { this.table = TABLES.create( - SCHEMA, SPEC, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), tableLocation); + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + tableLocation); } private void setupUnpartitionedTable() { @@ -227,11 +243,33 @@ private void setupUnpartitionedTable() { TABLES.create( SCHEMA, PartitionSpec.unpartitioned(), - ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), tableLocation); } - @Test + private DeleteFile fileADeletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_A) : FILE_A_POS_DELETES; + } + + private DeleteFile fileA2Deletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_A2) : FILE_A2_POS_DELETES; + } + + private DeleteFile fileBDeletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_B) : FILE_B_POS_DELETES; + } + + private DeleteFile fileB2Deletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_B2) : FILE_B2_POS_DELETES; + } + + private DeleteFile fileUnpartitionedDeletes() { + return formatVersion >= 3 + ? FileGenerationUtil.generateDV(table, FILE_UNPARTITIONED) + : FILE_UNPARTITIONED_POS_DELETE; + } + + @TestTemplate public void testPartitionedDeletesWithLesserSeqNo() { setupPartitionedTable(); @@ -239,12 +277,16 @@ public void testPartitionedDeletesWithLesserSeqNo() { table.newAppend().appendFile(FILE_B).appendFile(FILE_C).appendFile(FILE_D).commit(); // Add Delete Files + DeleteFile fileADeletes = fileADeletes(); + DeleteFile fileA2Deletes = fileA2Deletes(); + DeleteFile fileBDeletes = fileBDeletes(); + DeleteFile fileB2Deletes = fileB2Deletes(); table .newRowDelta() - .addDeletes(FILE_A_POS_DELETES) - .addDeletes(FILE_A2_POS_DELETES) - .addDeletes(FILE_B_POS_DELETES) - .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(fileADeletes) + .addDeletes(fileA2Deletes) + .addDeletes(fileBDeletes) + .addDeletes(fileB2Deletes) .addDeletes(FILE_A_EQ_DELETES) .addDeletes(FILE_A2_EQ_DELETES) .addDeletes(FILE_B_EQ_DELETES) @@ -275,18 +317,18 @@ public void testPartitionedDeletesWithLesserSeqNo() { Tuple2.apply(1L, FILE_C.path().toString()), Tuple2.apply(1L, FILE_D.path().toString()), Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileADeletes.path().toString()), Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileA2Deletes.path().toString()), Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileBDeletes.path().toString()), Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileB2Deletes.path().toString()), Tuple2.apply(3L, FILE_A2.path().toString()), Tuple2.apply(3L, FILE_B2.path().toString()), Tuple2.apply(3L, FILE_C2.path().toString()), Tuple2.apply(3L, FILE_D2.path().toString())); - assertThat(actual).isEqualTo(expected); + assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); RemoveDanglingDeleteFiles.Result result = SparkActions.get().removeDanglingDeleteFiles(table).execute(); @@ -302,8 +344,8 @@ public void testPartitionedDeletesWithLesserSeqNo() { .as("Expected 4 delete files removed") .hasSize(4) .containsExactlyInAnyOrder( - FILE_A_POS_DELETES.path(), - FILE_A2_POS_DELETES.path(), + fileADeletes.path(), + fileA2Deletes.path(), FILE_A_EQ_DELETES.path(), FILE_A2_EQ_DELETES.path()); @@ -323,17 +365,17 @@ public void testPartitionedDeletesWithLesserSeqNo() { Tuple2.apply(1L, FILE_C.path().toString()), Tuple2.apply(1L, FILE_D.path().toString()), Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileBDeletes.path().toString()), Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileB2Deletes.path().toString()), Tuple2.apply(3L, FILE_A2.path().toString()), Tuple2.apply(3L, FILE_B2.path().toString()), Tuple2.apply(3L, FILE_C2.path().toString()), Tuple2.apply(3L, FILE_D2.path().toString())); - assertThat(actualAfter).isEqualTo(expectedAfter); + assertThat(actualAfter).containsExactlyInAnyOrderElementsOf(expectedAfter); } - @Test + @TestTemplate public void testPartitionedDeletesWithEqSeqNo() { setupPartitionedTable(); @@ -341,18 +383,22 @@ public void testPartitionedDeletesWithEqSeqNo() { table.newAppend().appendFile(FILE_A).appendFile(FILE_C).appendFile(FILE_D).commit(); // Add Data Files with EQ and POS deletes + DeleteFile fileADeletes = fileADeletes(); + DeleteFile fileA2Deletes = fileA2Deletes(); + DeleteFile fileBDeletes = fileBDeletes(); + DeleteFile fileB2Deletes = fileB2Deletes(); table .newRowDelta() .addRows(FILE_A2) .addRows(FILE_B2) .addRows(FILE_C2) .addRows(FILE_D2) - .addDeletes(FILE_A_POS_DELETES) - .addDeletes(FILE_A2_POS_DELETES) + .addDeletes(fileADeletes) + .addDeletes(fileA2Deletes) .addDeletes(FILE_A_EQ_DELETES) .addDeletes(FILE_A2_EQ_DELETES) - .addDeletes(FILE_B_POS_DELETES) - .addDeletes(FILE_B2_POS_DELETES) + .addDeletes(fileBDeletes) + .addDeletes(fileB2Deletes) .addDeletes(FILE_B_EQ_DELETES) .addDeletes(FILE_B2_EQ_DELETES) .commit(); @@ -372,18 +418,18 @@ public void testPartitionedDeletesWithEqSeqNo() { Tuple2.apply(1L, FILE_C.path().toString()), Tuple2.apply(1L, FILE_D.path().toString()), Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileADeletes.path().toString()), Tuple2.apply(2L, FILE_A2.path().toString()), Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileA2Deletes.path().toString()), Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileBDeletes.path().toString()), Tuple2.apply(2L, FILE_B2.path().toString()), Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileB2Deletes.path().toString()), Tuple2.apply(2L, FILE_C2.path().toString()), Tuple2.apply(2L, FILE_D2.path().toString())); - assertThat(actual).isEqualTo(expected); + assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); RemoveDanglingDeleteFiles.Result result = SparkActions.get().removeDanglingDeleteFiles(table).execute(); @@ -415,25 +461,25 @@ public void testPartitionedDeletesWithEqSeqNo() { Tuple2.apply(1L, FILE_C.path().toString()), Tuple2.apply(1L, FILE_D.path().toString()), Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileADeletes.path().toString()), Tuple2.apply(2L, FILE_A2.path().toString()), Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileA2Deletes.path().toString()), + Tuple2.apply(2L, fileBDeletes.path().toString()), Tuple2.apply(2L, FILE_B2.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), + Tuple2.apply(2L, fileB2Deletes.path().toString()), Tuple2.apply(2L, FILE_C2.path().toString()), Tuple2.apply(2L, FILE_D2.path().toString())); - assertThat(actualAfter).isEqualTo(expectedAfter); + assertThat(actualAfter).containsExactlyInAnyOrderElementsOf(expectedAfter); } - @Test + @TestTemplate public void testUnpartitionedTable() { setupUnpartitionedTable(); table .newRowDelta() - .addDeletes(FILE_UNPARTITIONED_POS_DELETE) + .addDeletes(fileUnpartitionedDeletes()) .addDeletes(FILE_UNPARTITIONED_EQ_DELETE) .commit(); table.newAppend().appendFile(FILE_UNPARTITIONED).commit(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 12defafff06d..d36898d4c464 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -35,6 +36,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -45,6 +47,9 @@ import org.apache.iceberg.Files; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -81,9 +86,11 @@ import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestRemoveOrphanFilesAction extends TestBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); @@ -97,16 +104,23 @@ public abstract class TestRemoveOrphanFilesAction extends TestBase { @TempDir private File tableDir = null; protected String tableLocation = null; + protected Map properties; + @Parameter private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(2, 3); + } @BeforeEach public void setupTableLocation() throws Exception { this.tableLocation = tableDir.toURI().toString(); + properties = ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); } - @Test - public void testDryRun() throws IOException, InterruptedException { - Table table = - TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableLocation); + @TestTemplate + public void testDryRun() throws IOException { + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); @@ -184,9 +198,9 @@ public void testDryRun() throws IOException, InterruptedException { assertThat(actualRecords).isEqualTo(expectedRecords); } - @Test - public void testAllValidFilesAreKept() throws IOException, InterruptedException { - Table table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); + @TestTemplate + public void testAllValidFilesAreKept() throws IOException { + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); List records1 = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); @@ -246,9 +260,9 @@ public void testAllValidFilesAreKept() throws IOException, InterruptedException } } - @Test - public void orphanedFileRemovedWithParallelTasks() throws InterruptedException, IOException { - Table table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); + @TestTemplate + public void orphanedFileRemovedWithParallelTasks() { + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); List records1 = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); @@ -310,10 +324,12 @@ public void orphanedFileRemovedWithParallelTasks() throws InterruptedException, assertThat(deletedFiles).hasSize(4); } - @Test - public void testWapFilesAreKept() throws InterruptedException { + @TestTemplate + public void testWapFilesAreKept() { + assumeThat(formatVersion).as("currently fails with DVs").isEqualTo(2); Map props = Maps.newHashMap(); props.put(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, "true"); + props.putAll(properties); Table table = TABLES.create(SCHEMA, SPEC, props, tableLocation); List records = @@ -331,6 +347,8 @@ public void testWapFilesAreKept() throws InterruptedException { Dataset resultDF = spark.read().format("iceberg").load(tableLocation); List actualRecords = resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + + // TODO: currently fails because DVs delete stuff from WAP branch assertThat(actualRecords) .as("Should not return data from the staged snapshot") .isEqualTo(records); @@ -345,11 +363,12 @@ public void testWapFilesAreKept() throws InterruptedException { assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); } - @Test - public void testMetadataFolderIsIntact() throws InterruptedException { + @TestTemplate + public void testMetadataFolderIsIntact() { // write data directly to the table location Map props = Maps.newHashMap(); props.put(TableProperties.WRITE_DATA_LOCATION, tableLocation); + props.putAll(properties); Table table = TABLES.create(SCHEMA, SPEC, props, tableLocation); List records = @@ -375,9 +394,9 @@ public void testMetadataFolderIsIntact() throws InterruptedException { assertThat(actualRecords).as("Rows must match").isEqualTo(records); } - @Test - public void testOlderThanTimestamp() throws InterruptedException { - Table table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); + @TestTemplate + public void testOlderThanTimestamp() { + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); @@ -404,11 +423,12 @@ public void testOlderThanTimestamp() throws InterruptedException { assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); } - @Test - public void testRemoveUnreachableMetadataVersionFiles() throws InterruptedException { + @TestTemplate + public void testRemoveUnreachableMetadataVersionFiles() { Map props = Maps.newHashMap(); props.put(TableProperties.WRITE_DATA_LOCATION, tableLocation); props.put(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, "1"); + props.putAll(properties); Table table = TABLES.create(SCHEMA, SPEC, props, tableLocation); List records = @@ -441,9 +461,9 @@ public void testRemoveUnreachableMetadataVersionFiles() throws InterruptedExcept assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); } - @Test - public void testManyTopLevelPartitions() throws InterruptedException { - Table table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); + @TestTemplate + public void testManyTopLevelPartitions() { + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); List records = Lists.newArrayList(); for (int i = 0; i < 100; i++) { @@ -467,9 +487,9 @@ public void testManyTopLevelPartitions() throws InterruptedException { assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); } - @Test - public void testManyLeafPartitions() throws InterruptedException { - Table table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); + @TestTemplate + public void testManyLeafPartitions() { + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); List records = Lists.newArrayList(); for (int i = 0; i < 100; i++) { @@ -493,15 +513,15 @@ public void testManyLeafPartitions() throws InterruptedException { assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); } - @Test - public void testHiddenPartitionPaths() throws InterruptedException { + @TestTemplate + public void testHiddenPartitionPaths() { Schema schema = new Schema( optional(1, "c1", Types.IntegerType.get()), optional(2, "_c2", Types.StringType.get()), optional(3, "c3", Types.StringType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("_c2", 2).identity("c3").build(); - Table table = TABLES.create(schema, spec, Maps.newHashMap(), tableLocation); + Table table = TABLES.create(schema, spec, properties, tableLocation); StructType structType = new StructType() @@ -526,15 +546,15 @@ public void testHiddenPartitionPaths() throws InterruptedException { assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); } - @Test - public void testHiddenPartitionPathsWithPartitionEvolution() throws InterruptedException { + @TestTemplate + public void testHiddenPartitionPathsWithPartitionEvolution() { Schema schema = new Schema( optional(1, "_c1", Types.IntegerType.get()), optional(2, "_c2", Types.StringType.get()), optional(3, "c3", Types.StringType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("_c2", 2).build(); - Table table = TABLES.create(schema, spec, Maps.newHashMap(), tableLocation); + Table table = TABLES.create(schema, spec, properties, tableLocation); StructType structType = new StructType() @@ -562,16 +582,15 @@ public void testHiddenPartitionPathsWithPartitionEvolution() throws InterruptedE assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); } - @Test - public void testHiddenPathsStartingWithPartitionNamesAreIgnored() - throws InterruptedException, IOException { + @TestTemplate + public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOException { Schema schema = new Schema( optional(1, "c1", Types.IntegerType.get()), optional(2, "_c2", Types.StringType.get()), optional(3, "c3", Types.StringType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("_c2", 2).identity("c3").build(); - Table table = TABLES.create(schema, spec, Maps.newHashMap(), tableLocation); + Table table = TABLES.create(schema, spec, properties, tableLocation); StructType structType = new StructType() @@ -610,11 +629,11 @@ private List snapshotFiles(long snapshotId) { .collectAsList(); } - @Test - public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException, InterruptedException { + @TestTemplate + public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { Table table = TABLES.create( - SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableDir.getAbsolutePath()); + SCHEMA, PartitionSpec.unpartitioned(), properties, tableDir.getAbsolutePath()); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); @@ -670,7 +689,7 @@ public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException, Inte .isTrue(); } - @Test + @TestTemplate public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException { HadoopCatalog catalog = new HadoopCatalog(new Configuration(), tableLocation); String namespaceName = "testDb"; @@ -705,15 +724,11 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException assertThat(actualRecords).as("Rows must match").isEqualTo(records); } - @Test + @TestTemplate public void testHiveCatalogTable() throws IOException { - Table table = - catalog.createTable( - TableIdentifier.of("default", "hivetestorphan"), - SCHEMA, - SPEC, - tableLocation, - Maps.newHashMap()); + TableIdentifier identifier = + TableIdentifier.of("default", "hivetestorphan" + ThreadLocalRandom.current().nextInt(1000)); + Table table = catalog.createTable(identifier, SCHEMA, SPEC, tableLocation, properties); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); @@ -724,7 +739,7 @@ public void testHiveCatalogTable() throws IOException { .write() .format("iceberg") .mode("append") - .save("default.hivetestorphan"); + .save(identifier.toString()); String location = table.location().replaceFirst("file:", ""); new File(location + "/data/trashfile").createNewFile(); @@ -739,10 +754,9 @@ public void testHiveCatalogTable() throws IOException { .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); } - @Test + @TestTemplate public void testGarbageCollectionDisabled() { - Table table = - TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableLocation); + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); @@ -759,10 +773,9 @@ public void testGarbageCollectionDisabled() { "Cannot delete orphan files: GC is disabled (deleting files may corrupt other tables)"); } - @Test - public void testCompareToFileList() throws IOException, InterruptedException { - Table table = - TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableLocation); + @TestTemplate + public void testCompareToFileList() throws IOException { + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); @@ -890,14 +903,10 @@ protected long waitUntilAfter(long timestampMillis) { return current; } - @Test + @TestTemplate public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { - Table table = - TABLES.create( - SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), - tableLocation); + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); List records = Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); @@ -966,28 +975,28 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); } - @Test + @TestTemplate public void testPathsWithExtraSlashes() { List validFiles = Lists.newArrayList("file:///dir1/dir2/file1"); List actualFiles = Lists.newArrayList("file:///dir1/////dir2///file1"); executeTest(validFiles, actualFiles, Lists.newArrayList()); } - @Test + @TestTemplate public void testPathsWithValidFileHavingNoAuthority() { List validFiles = Lists.newArrayList("hdfs:///dir1/dir2/file1"); List actualFiles = Lists.newArrayList("hdfs://servicename/dir1/dir2/file1"); executeTest(validFiles, actualFiles, Lists.newArrayList()); } - @Test + @TestTemplate public void testPathsWithActualFileHavingNoAuthority() { List validFiles = Lists.newArrayList("hdfs://servicename/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("hdfs:///dir1/dir2/file1"); executeTest(validFiles, actualFiles, Lists.newArrayList()); } - @Test + @TestTemplate public void testPathsWithEqualSchemes() { List validFiles = Lists.newArrayList("scheme1://bucket1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("scheme2://bucket1/dir1/dir2/file1"); @@ -1016,7 +1025,7 @@ public void testPathsWithEqualSchemes() { DeleteOrphanFiles.PrefixMismatchMode.ERROR); } - @Test + @TestTemplate public void testPathsWithEqualAuthorities() { List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); @@ -1045,7 +1054,7 @@ public void testPathsWithEqualAuthorities() { DeleteOrphanFiles.PrefixMismatchMode.ERROR); } - @Test + @TestTemplate public void testRemoveOrphanFileActionWithDeleteMode() { List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 2476d1bb7078..14784da4f74f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -21,10 +21,9 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.File; -import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; import java.util.stream.StreamSupport; import org.apache.iceberg.actions.DeleteOrphanFiles; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.SparkCatalog; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkSessionCatalog; @@ -32,10 +31,10 @@ import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.expressions.Transform; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; public class TestRemoveOrphanFilesAction3 extends TestRemoveOrphanFilesAction { - @Test + @TestTemplate public void testSparkCatalogTable() throws Exception { spark.conf().set("spark.sql.catalog.mycat", "org.apache.iceberg.spark.SparkCatalog"); spark.conf().set("spark.sql.catalog.mycat.type", "hadoop"); @@ -43,16 +42,16 @@ public void testSparkCatalogTable() throws Exception { SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("mycat"); String[] database = {"default"}; - Identifier id = Identifier.of(database, "table"); - Map options = Maps.newHashMap(); + Identifier id = Identifier.of(database, "table" + ThreadLocalRandom.current().nextInt(1000)); Transform[] transforms = {}; - cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options); + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); SparkTable table = (SparkTable) cat.loadTable(id); - spark.sql("INSERT INTO mycat.default.table VALUES (1,1,1)"); + sql("INSERT INTO mycat.default.%s VALUES (1,1,1)", id.name()); String location = table.table().location().replaceFirst("file:", ""); - new File(location + "/data/trashfile").createNewFile(); + String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = SparkActions.get() @@ -61,10 +60,10 @@ public void testSparkCatalogTable() throws Exception { .execute(); assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); + .anyMatch(file -> file.contains("file:" + location + trashFile)); } - @Test + @TestTemplate public void testSparkCatalogNamedHadoopTable() throws Exception { spark.conf().set("spark.sql.catalog.hadoop", "org.apache.iceberg.spark.SparkCatalog"); spark.conf().set("spark.sql.catalog.hadoop.type", "hadoop"); @@ -72,16 +71,16 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("hadoop"); String[] database = {"default"}; - Identifier id = Identifier.of(database, "table"); - Map options = Maps.newHashMap(); + Identifier id = Identifier.of(database, "table" + ThreadLocalRandom.current().nextInt(1000)); Transform[] transforms = {}; - cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options); + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); SparkTable table = (SparkTable) cat.loadTable(id); - spark.sql("INSERT INTO hadoop.default.table VALUES (1,1,1)"); + sql("INSERT INTO hadoop.default.%s VALUES (1,1,1)", id.name()); String location = table.table().location().replaceFirst("file:", ""); - new File(location + "/data/trashfile").createNewFile(); + String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = SparkActions.get() @@ -90,10 +89,10 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { .execute(); assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); + .anyMatch(file -> file.contains("file:" + location + trashFile)); } - @Test + @TestTemplate public void testSparkCatalogNamedHiveTable() throws Exception { spark.conf().set("spark.sql.catalog.hive", "org.apache.iceberg.spark.SparkCatalog"); spark.conf().set("spark.sql.catalog.hive.type", "hadoop"); @@ -101,16 +100,16 @@ public void testSparkCatalogNamedHiveTable() throws Exception { SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("hive"); String[] database = {"default"}; - Identifier id = Identifier.of(database, "table"); - Map options = Maps.newHashMap(); + Identifier id = Identifier.of(database, "table" + ThreadLocalRandom.current().nextInt(1000)); Transform[] transforms = {}; - cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options); + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); SparkTable table = (SparkTable) cat.loadTable(id); - spark.sql("INSERT INTO hive.default.table VALUES (1,1,1)"); + sql("INSERT INTO hive.default.%s VALUES (1,1,1)", id.name()); String location = table.table().location().replaceFirst("file:", ""); - new File(location + "/data/trashfile").createNewFile(); + String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = SparkActions.get() @@ -120,10 +119,10 @@ public void testSparkCatalogNamedHiveTable() throws Exception { assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); + .anyMatch(file -> file.contains("file:" + location + trashFile)); } - @Test + @TestTemplate public void testSparkSessionCatalogHadoopTable() throws Exception { spark .conf() @@ -134,16 +133,16 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { (SparkSessionCatalog) spark.sessionState().catalogManager().v2SessionCatalog(); String[] database = {"default"}; - Identifier id = Identifier.of(database, "table"); - Map options = Maps.newHashMap(); + Identifier id = Identifier.of(database, "table" + ThreadLocalRandom.current().nextInt(1000)); Transform[] transforms = {}; - cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options); + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); SparkTable table = (SparkTable) cat.loadTable(id); - spark.sql("INSERT INTO default.table VALUES (1,1,1)"); + sql("INSERT INTO default.%s VALUES (1,1,1)", id.name()); String location = table.table().location().replaceFirst("file:", ""); - new File(location + "/data/trashfile").createNewFile(); + String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = SparkActions.get() @@ -152,10 +151,10 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { .execute(); assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); + .anyMatch(file -> file.contains("file:" + location + trashFile)); } - @Test + @TestTemplate public void testSparkSessionCatalogHiveTable() throws Exception { spark .conf() @@ -166,16 +165,16 @@ public void testSparkSessionCatalogHiveTable() throws Exception { String[] database = {"default"}; Identifier id = Identifier.of(database, "sessioncattest"); - Map options = Maps.newHashMap(); Transform[] transforms = {}; cat.dropTable(id); - cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options); + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); SparkTable table = (SparkTable) cat.loadTable(id); spark.sql("INSERT INTO default.sessioncattest VALUES (1,1,1)"); String location = table.table().location().replaceFirst("file:", ""); - new File(location + "/data/trashfile").createNewFile(); + String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = SparkActions.get() @@ -184,7 +183,7 @@ public void testSparkSessionCatalogHiveTable() throws Exception { .execute(); assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) .as("trash file should be removed") - .anyMatch(file -> file.contains("file:" + location + "/data/trashfile")); + .anyMatch(file -> file.contains("file:" + location + trashFile)); } @AfterEach diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index b7ab47f865b5..38c4d32a90d2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -27,6 +27,7 @@ import static org.apache.spark.sql.functions.min; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.doAnswer; @@ -38,7 +39,7 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; -import java.nio.file.Path; +import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -56,6 +57,9 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; @@ -77,6 +81,8 @@ import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteWriter; @@ -96,7 +102,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.spark.FileRewriteCoordinator; import org.apache.iceberg.spark.ScanTaskSetManager; @@ -119,11 +124,13 @@ import org.apache.spark.sql.internal.SQLConf; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; import org.mockito.ArgumentMatcher; import org.mockito.Mockito; +@ExtendWith(ParameterizedTestExtension.class) public class TestRewriteDataFilesAction extends TestBase { @TempDir private File tableDir; @@ -138,7 +145,12 @@ public class TestRewriteDataFilesAction extends TestBase { private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); - @TempDir private Path temp; + @Parameter private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(2, 3); + } private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); private final ScanTaskSetManager manager = ScanTaskSetManager.get(); @@ -161,10 +173,11 @@ private RewriteDataFilesSparkAction basicRewrite(Table table) { return actions().rewriteDataFiles(table).option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1"); } - @Test + @TestTemplate public void testEmptyTable() { PartitionSpec spec = PartitionSpec.unpartitioned(); - Map options = Maps.newHashMap(); + Map options = + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); Table table = TABLES.create(SCHEMA, spec, options, tableLocation); assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); @@ -174,7 +187,7 @@ public void testEmptyTable() { assertThat(table.currentSnapshot()).as("Table must stay empty").isNull(); } - @Test + @TestTemplate public void testBinPackUnpartitionedTable() { Table table = createTable(4); shouldHaveFiles(table, 4); @@ -194,7 +207,7 @@ public void testBinPackUnpartitionedTable() { assertEquals("Rows must match", expectedRecords, actual); } - @Test + @TestTemplate public void testBinPackPartitionedTable() { Table table = createTablePartitioned(4, 2); shouldHaveFiles(table, 8); @@ -214,7 +227,7 @@ public void testBinPackPartitionedTable() { assertEquals("Rows must match", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testBinPackWithFilter() { Table table = createTablePartitioned(4, 2); shouldHaveFiles(table, 8); @@ -239,7 +252,7 @@ public void testBinPackWithFilter() { assertEquals("Rows must match", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testBinPackWithFilterOnBucketExpression() { Table table = createTablePartitioned(4, 2); @@ -265,7 +278,7 @@ public void testBinPackWithFilterOnBucketExpression() { assertEquals("Rows must match", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testBinPackAfterPartitionChange() { Table table = createTable(); @@ -300,10 +313,10 @@ public void testBinPackAfterPartitionChange() { shouldHaveFiles(table, 20); } - @Test - public void testBinPackWithDeletes() { + @TestTemplate + public void testBinPackWithDeletes() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); Table table = createTablePartitioned(4, 2); - table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); shouldHaveFiles(table, 8); table.refresh(); @@ -311,14 +324,28 @@ public void testBinPackWithDeletes() { int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); RowDelta rowDelta = table.newRowDelta(); - // add 1 delete file for data files 0, 1, 2 - for (int i = 0; i < 3; i++) { - writePosDeletesToFile(table, dataFiles.get(i), 1).forEach(rowDelta::addDeletes); - } + if (formatVersion >= 3) { + // delete 1 position for data files 0, 1, 2 + for (int i = 0; i < 3; i++) { + writeDV(table, dataFiles.get(i).partition(), dataFiles.get(i).location(), 1) + .forEach(rowDelta::addDeletes); + } - // add 2 delete files for data files 3, 4 - for (int i = 3; i < 5; i++) { - writePosDeletesToFile(table, dataFiles.get(i), 2).forEach(rowDelta::addDeletes); + // delete 2 positions for data files 3, 4 + for (int i = 3; i < 5; i++) { + writeDV(table, dataFiles.get(i).partition(), dataFiles.get(i).location(), 2) + .forEach(rowDelta::addDeletes); + } + } else { + // add 1 delete file for data files 0, 1, 2 + for (int i = 0; i < 3; i++) { + writePosDeletesToFile(table, dataFiles.get(i), 1).forEach(rowDelta::addDeletes); + } + + // add 2 delete files for data files 3, 4 + for (int i = 3; i < 5; i++) { + writePosDeletesToFile(table, dataFiles.get(i), 2).forEach(rowDelta::addDeletes); + } } rowDelta.commit(); @@ -326,32 +353,49 @@ public void testBinPackWithDeletes() { List expectedRecords = currentData(); long dataSizeBefore = testDataSize(table); - Result result = - actions() - .rewriteDataFiles(table) - // do not include any file based on bin pack file size configs - .option(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "0") - .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) - .option(SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) - .option(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "2") - .execute(); - assertThat(result.rewrittenDataFilesCount()) - .as("Action should rewrite 2 data files") - .isEqualTo(2); - assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + if (formatVersion >= 3) { + Result result = + actions() + .rewriteDataFiles(table) + // do not include any file based on bin pack file size configs + .option(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "0") + .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .option(SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) + // set DELETE_FILE_THRESHOLD to 1 since DVs only produce one delete file per data file + .option(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "1") + .execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 5 data files") + .isEqualTo(5); + assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + } else { + Result result = + actions() + .rewriteDataFiles(table) + // do not include any file based on bin pack file size configs + .option(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "0") + .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .option(SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) + .option(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "2") + .execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 2 data files") + .isEqualTo(2); + assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + } List actualRecords = currentData(); assertEquals("Rows must match", expectedRecords, actualRecords); assertThat(actualRecords).as("7 rows are removed").hasSize(total - 7); } - @Test + @TestTemplate public void testRemoveDangledEqualityDeletesPartitionEvolution() { Table table = TABLES.create( SCHEMA, SPEC, - Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), tableLocation); // data seq = 1, write 4 files in 2 partitions @@ -414,13 +458,13 @@ public void testRemoveDangledEqualityDeletesPartitionEvolution() { shouldHaveFiles(table, 5); } - @Test - public void testRemoveDangledPositionDeletesPartitionEvolution() { + @TestTemplate + public void testRemoveDangledPositionDeletesPartitionEvolution() throws IOException { Table table = TABLES.create( SCHEMA, SPEC, - Collections.singletonMap(TableProperties.FORMAT_VERSION, "2"), + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), tableLocation); // data seq = 1, write 4 files in 2 partitions @@ -428,11 +472,15 @@ public void testRemoveDangledPositionDeletesPartitionEvolution() { List dataFilesBefore = TestHelpers.dataFiles(table, null); shouldHaveFiles(table, 4); + DeleteFile deleteFile; // data seq = 2, write 1 position deletes in c1=1 - table - .newRowDelta() - .addDeletes(writePosDeletesToFile(table, dataFilesBefore.get(3), 1).get(0)) - .commit(); + DataFile dataFile = dataFilesBefore.get(3); + if (formatVersion >= 3) { + deleteFile = writeDV(table, dataFile.partition(), dataFile.location(), 1).get(0); + } else { + deleteFile = writePosDeletesToFile(table, dataFile, 1).get(0); + } + table.newRowDelta().addDeletes(deleteFile).commit(); // partition evolution table.updateSpec().addField(Expressions.ref("c3")).commit(); @@ -464,11 +512,10 @@ public void testRemoveDangledPositionDeletesPartitionEvolution() { assertEquals("Rows must match", expectedRecords, currentData()); } - @Test - public void testBinPackWithDeleteAllData() { - Map options = Maps.newHashMap(); - options.put(TableProperties.FORMAT_VERSION, "2"); - Table table = createTablePartitioned(1, 1, 1, options); + @TestTemplate + public void testBinPackWithDeleteAllData() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + Table table = createTablePartitioned(1, 1, 1); shouldHaveFiles(table, 1); table.refresh(); @@ -476,8 +523,14 @@ public void testBinPackWithDeleteAllData() { int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); RowDelta rowDelta = table.newRowDelta(); + DataFile dataFile = dataFiles.get(0); // remove all data - writePosDeletesToFile(table, dataFiles.get(0), total).forEach(rowDelta::addDeletes); + if (formatVersion >= 3) { + writeDV(table, dataFile.partition(), dataFile.location(), total) + .forEach(rowDelta::addDeletes); + } else { + writePosDeletesToFile(table, dataFile, total).forEach(rowDelta::addDeletes); + } rowDelta.commit(); table.refresh(); @@ -507,12 +560,12 @@ public void testBinPackWithDeleteAllData() { .isEqualTo(total); } - @Test + @TestTemplate public void testBinPackWithStartingSequenceNumber() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); Table table = createTablePartitioned(4, 2); shouldHaveFiles(table, 8); List expectedRecords = currentData(); - table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); table.refresh(); long oldSequenceNumber = table.currentSnapshot().sequenceNumber(); long dataSizeBefore = testDataSize(table); @@ -544,7 +597,7 @@ public void testBinPackWithStartingSequenceNumber() { } } - @Test + @TestTemplate public void testBinPackWithStartingSequenceNumberV1Compatibility() { Map properties = ImmutableMap.of(TableProperties.FORMAT_VERSION, "1"); Table table = createTablePartitioned(4, 2, SCALE, properties); @@ -580,11 +633,15 @@ public void testBinPackWithStartingSequenceNumberV1Compatibility() { } } - @Test + @TestTemplate public void testRewriteLargeTableHasResiduals() { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).build(); - Map options = Maps.newHashMap(); - options.put(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100"); + Map options = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, + "100"); Table table = TABLES.create(SCHEMA, spec, options, tableLocation); // all records belong to the same partition @@ -622,7 +679,7 @@ public void testRewriteLargeTableHasResiduals() { assertEquals("Rows must match", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testBinPackSplitLargeFile() { Table table = createTable(1); shouldHaveFiles(table, 1); @@ -647,7 +704,7 @@ public void testBinPackSplitLargeFile() { assertEquals("Rows must match", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testBinPackCombineMixedFiles() { Table table = createTable(1); // 400000 shouldHaveFiles(table, 1); @@ -683,7 +740,7 @@ public void testBinPackCombineMixedFiles() { assertEquals("Rows must match", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testBinPackCombineMediumFiles() { Table table = createTable(4); shouldHaveFiles(table, 4); @@ -716,7 +773,7 @@ public void testBinPackCombineMediumFiles() { assertEquals("Rows must match", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testPartialProgressEnabled() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -747,7 +804,7 @@ public void testPartialProgressEnabled() { assertEquals("We shouldn't have changed the data", originalData, postRewriteData); } - @Test + @TestTemplate public void testMultipleGroups() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -775,7 +832,7 @@ public void testMultipleGroups() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testPartialProgressMaxCommits() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -804,7 +861,7 @@ public void testPartialProgressMaxCommits() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testSingleCommitWithRewriteFailure() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -838,7 +895,7 @@ public void testSingleCommitWithRewriteFailure() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testSingleCommitWithCommitFailure() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -872,7 +929,7 @@ public void testSingleCommitWithCommitFailure() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testCommitFailsWithUncleanableFailure() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -906,7 +963,7 @@ public void testCommitFailsWithUncleanableFailure() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testParallelSingleCommitWithRewriteFailure() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -941,7 +998,7 @@ public void testParallelSingleCommitWithRewriteFailure() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testPartialProgressWithRewriteFailure() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -983,7 +1040,7 @@ public void testPartialProgressWithRewriteFailure() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testParallelPartialProgressWithRewriteFailure() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -1026,7 +1083,7 @@ public void testParallelPartialProgressWithRewriteFailure() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testParallelPartialProgressWithCommitFailure() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -1071,7 +1128,7 @@ public void testParallelPartialProgressWithCommitFailure() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testParallelPartialProgressWithMaxFailedCommits() { Table table = createTable(20); int fileSize = averageFileSize(table); @@ -1113,7 +1170,7 @@ public void testParallelPartialProgressWithMaxFailedCommits() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testInvalidOptions() { Table table = createTable(20); @@ -1157,7 +1214,7 @@ public void testInvalidOptions() { .hasMessageContaining("requires enabling Iceberg Spark session extensions"); } - @Test + @TestTemplate public void testSortMultipleGroups() { Table table = createTable(20); shouldHaveFiles(table, 20); @@ -1189,7 +1246,7 @@ public void testSortMultipleGroups() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testSimpleSort() { Table table = createTable(20); shouldHaveFiles(table, 20); @@ -1222,7 +1279,7 @@ public void testSimpleSort() { shouldHaveLastCommitSorted(table, "c2"); } - @Test + @TestTemplate public void testSortAfterPartitionChange() { Table table = createTable(20); shouldHaveFiles(table, 20); @@ -1258,7 +1315,7 @@ public void testSortAfterPartitionChange() { shouldHaveLastCommitSorted(table, "c2"); } - @Test + @TestTemplate public void testSortCustomSortOrder() { Table table = createTable(20); shouldHaveLastCommitUnsorted(table, "c2"); @@ -1289,7 +1346,7 @@ public void testSortCustomSortOrder() { shouldHaveLastCommitSorted(table, "c2"); } - @Test + @TestTemplate public void testSortCustomSortOrderRequiresRepartition() { int partitions = 4; Table table = createTable(); @@ -1329,7 +1386,7 @@ public void testSortCustomSortOrderRequiresRepartition() { shouldHaveLastCommitSorted(table, "c3"); } - @Test + @TestTemplate public void testAutoSortShuffleOutput() { Table table = createTable(20); shouldHaveLastCommitUnsorted(table, "c2"); @@ -1369,7 +1426,7 @@ public void testAutoSortShuffleOutput() { shouldHaveLastCommitSorted(table, "c2"); } - @Test + @TestTemplate public void testCommitStateUnknownException() { Table table = createTable(20); shouldHaveFiles(table, 20); @@ -1401,7 +1458,7 @@ public void testCommitStateUnknownException() { shouldHaveSnapshots(table, 2); // Commit actually Succeeded } - @Test + @TestTemplate public void testZOrderSort() { int originalFiles = 20; Table table = createTable(originalFiles); @@ -1461,7 +1518,7 @@ public void testZOrderSort() { .isGreaterThan(filesScannedC2C3); } - @Test + @TestTemplate public void testZOrderAllTypesSort() { Table table = createTypeTestTable(); shouldHaveFiles(table, 10); @@ -1505,7 +1562,7 @@ public void testZOrderAllTypesSort() { shouldHaveACleanCache(table); } - @Test + @TestTemplate public void testInvalidAPIUsage() { Table table = createTable(1); @@ -1524,14 +1581,14 @@ public void testInvalidAPIUsage() { .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); } - @Test + @TestTemplate public void testRewriteJobOrderBytesAsc() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); Table table = createTablePartitioned(4, 2); writeRecords(1, SCALE, 1); writeRecords(2, SCALE, 2); writeRecords(3, SCALE, 3); writeRecords(4, SCALE, 4); - table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); RewriteDataFilesSparkAction basicRewrite = basicRewrite(table).binPack(); List expected = @@ -1556,14 +1613,14 @@ public void testRewriteJobOrderBytesAsc() { assertThat(actual).as("Size in bytes order should not be descending").isNotEqualTo(expected); } - @Test + @TestTemplate public void testRewriteJobOrderBytesDesc() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); Table table = createTablePartitioned(4, 2); writeRecords(1, SCALE, 1); writeRecords(2, SCALE, 2); writeRecords(3, SCALE, 3); writeRecords(4, SCALE, 4); - table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); RewriteDataFilesSparkAction basicRewrite = basicRewrite(table).binPack(); List expected = @@ -1588,14 +1645,14 @@ public void testRewriteJobOrderBytesDesc() { assertThat(actual).as("Size in bytes order should not be ascending").isNotEqualTo(expected); } - @Test + @TestTemplate public void testRewriteJobOrderFilesAsc() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); Table table = createTablePartitioned(4, 2); writeRecords(1, SCALE, 1); writeRecords(2, SCALE, 2); writeRecords(3, SCALE, 3); writeRecords(4, SCALE, 4); - table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); RewriteDataFilesSparkAction basicRewrite = basicRewrite(table).binPack(); List expected = @@ -1620,14 +1677,14 @@ public void testRewriteJobOrderFilesAsc() { assertThat(actual).as("Number of files order should not be descending").isNotEqualTo(expected); } - @Test + @TestTemplate public void testRewriteJobOrderFilesDesc() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); Table table = createTablePartitioned(4, 2); writeRecords(1, SCALE, 1); writeRecords(2, SCALE, 2); writeRecords(3, SCALE, 3); writeRecords(4, SCALE, 4); - table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); RewriteDataFilesSparkAction basicRewrite = basicRewrite(table).binPack(); List expected = @@ -1652,7 +1709,7 @@ public void testRewriteJobOrderFilesDesc() { assertThat(actual).as("Number of files order should not be ascending").isNotEqualTo(expected); } - @Test + @TestTemplate public void testSnapshotProperty() { Table table = createTable(4); Result ignored = basicRewrite(table).snapshotProperty("key", "value").execute(); @@ -1669,7 +1726,7 @@ public void testSnapshotProperty() { assertThat(table.currentSnapshot().summary()).containsKeys(commitMetricsKeys); } - @Test + @TestTemplate public void testBinPackRewriterWithSpecificUnparitionedOutputSpec() { Table table = createTable(10); shouldHaveFiles(table, 10); @@ -1691,7 +1748,7 @@ public void testBinPackRewriterWithSpecificUnparitionedOutputSpec() { shouldRewriteDataFilesWithPartitionSpec(table, outputSpecId); } - @Test + @TestTemplate public void testBinPackRewriterWithSpecificOutputSpec() { Table table = createTable(10); shouldHaveFiles(table, 10); @@ -1714,7 +1771,7 @@ public void testBinPackRewriterWithSpecificOutputSpec() { shouldRewriteDataFilesWithPartitionSpec(table, outputSpecId); } - @Test + @TestTemplate public void testBinpackRewriteWithInvalidOutputSpecId() { Table table = createTable(10); shouldHaveFiles(table, 10); @@ -1730,7 +1787,7 @@ public void testBinpackRewriteWithInvalidOutputSpecId() { "Cannot use output spec id 1234 because the table does not contain a reference to this spec-id."); } - @Test + @TestTemplate public void testSortRewriterWithSpecificOutputSpecId() { Table table = createTable(10); shouldHaveFiles(table, 10); @@ -1753,7 +1810,7 @@ public void testSortRewriterWithSpecificOutputSpecId() { shouldRewriteDataFilesWithPartitionSpec(table, outputSpecId); } - @Test + @TestTemplate public void testZOrderRewriteWithSpecificOutputSpecId() { Table table = createTable(10); shouldHaveFiles(table, 10); @@ -1956,7 +2013,8 @@ private List, Pair>> checkForOverlappingFiles( protected Table createTable() { PartitionSpec spec = PartitionSpec.unpartitioned(); - Map options = Maps.newHashMap(); + Map options = + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); Table table = TABLES.create(SCHEMA, spec, options, tableLocation); table .updateProperties() @@ -1989,7 +2047,19 @@ protected Table createTablePartitioned( } protected Table createTablePartitioned(int partitions, int files) { - return createTablePartitioned(partitions, files, SCALE, Maps.newHashMap()); + return createTablePartitioned( + partitions, + files, + SCALE, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion))); + } + + protected Table createTablePartitioned(int partitions, int files, int numRecords) { + return createTablePartitioned( + partitions, + files, + numRecords, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion))); } private Table createTypeTestTable() { @@ -2005,7 +2075,8 @@ private Table createTypeTestTable() { optional(8, "booleanCol", Types.BooleanType.get()), optional(9, "binaryCol", Types.BinaryType.get())); - Map options = Maps.newHashMap(); + Map options = + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); Table table = TABLES.create(schema, PartitionSpec.unpartitioned(), options, tableLocation); spark @@ -2123,6 +2194,20 @@ private List writePosDeletes( return results; } + private List writeDV( + Table table, StructLike partition, String path, int numPositionsToDelete) throws IOException { + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PUFFIN).build(); + DVFileWriter writer = new BaseDVFileWriter(fileFactory, p -> null); + try (DVFileWriter closeableWriter = writer) { + for (int row = 0; row < numPositionsToDelete; row++) { + closeableWriter.delete(path, row, table.spec(), partition); + } + } + + return writer.result().deleteFiles(); + } + private void writeEqDeleteRecord( Table table, String partCol, Object partVal, String delCol, Object delVal) { List equalityFieldIds = Lists.newArrayList(table.schema().findField(delCol).fieldId()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 11d61e599eba..44971843547b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -37,6 +37,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -107,6 +108,7 @@ public static Object[] parameters() { new Object[] {"false", "true", true, 1}, new Object[] {"true", "false", false, 2}, new Object[] {"false", "false", false, 2}, + new Object[] {"true", "false", false, 3}, new Object[] {"false", "false", false, 3} }; } @@ -1108,13 +1110,23 @@ private DeleteFile newDV(Table table, DataFile dataFile) { } private DeleteFile newDeleteFile(Table table, String partitionPath) { - return FileMetadata.deleteFileBuilder(table.spec()) - .ofPositionDeletes() - .withPath("/path/to/pos-deletes-" + UUID.randomUUID() + ".parquet") - .withFileSizeInBytes(5) - .withPartitionPath(partitionPath) - .withRecordCount(1) - .build(); + return formatVersion >= 3 + ? FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withPath("/path/to/pos-deletes-" + UUID.randomUUID() + ".puffin") + .withFileSizeInBytes(5) + .withPartitionPath(partitionPath) + .withRecordCount(1) + .withContentOffset(ThreadLocalRandom.current().nextInt()) + .withContentSizeInBytes(ThreadLocalRandom.current().nextInt()) + .build() + : FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withPath("/path/to/pos-deletes-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(5) + .withPartitionPath(partitionPath) + .withRecordCount(1) + .build(); } private List> generatePosDeletes(String predicate) { @@ -1145,7 +1157,7 @@ private Pair writePosDeletes( Table table, StructLike partition, List> deletes) throws IOException { OutputFile outputFile = Files.localOutput(File.createTempFile("junit", null, temp.toFile())); - return FileHelpers.writeDeleteFile(table, outputFile, partition, deletes); + return FileHelpers.writeDeleteFile(table, outputFile, partition, deletes, formatVersion); } private DeleteFile writeEqDeletes(Table table, String key, Object... values) throws IOException { From 3659ded18d50206576985339bd55cd82f5e200cc Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 13 Nov 2024 13:09:31 -0800 Subject: [PATCH 136/313] Spark: Update tests which assume file format to use enum instead of string literal (#11540) --- .../org/apache/iceberg/spark/source/TestSparkReaderDeletes.java | 2 +- .../org/apache/iceberg/spark/source/TestSparkReaderDeletes.java | 2 +- .../org/apache/iceberg/spark/source/TestSparkReaderDeletes.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index bde87778ad62..5ba296e74c35 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -545,7 +545,7 @@ public void testIsDeletedColumnWithoutDeleteFile() { @TestTemplate public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOException { - assumeThat(format).isEqualTo("parquet"); + assumeThat(format).isEqualTo(FileFormat.PARQUET); String tblName = "test3"; Table tbl = createTable(tblName, SCHEMA, PartitionSpec.unpartitioned()); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 29c2d4b39a1e..285a210db5f2 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -554,7 +554,7 @@ public void testIsDeletedColumnWithoutDeleteFile() { @TestTemplate public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOException { - assumeThat(format).isEqualTo("parquet"); + assumeThat(format).isEqualTo(FileFormat.PARQUET); String tblName = "test3"; Table tbl = createTable(tblName, SCHEMA, PartitionSpec.unpartitioned()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 29c2d4b39a1e..285a210db5f2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -554,7 +554,7 @@ public void testIsDeletedColumnWithoutDeleteFile() { @TestTemplate public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOException { - assumeThat(format).isEqualTo("parquet"); + assumeThat(format).isEqualTo(FileFormat.PARQUET); String tblName = "test3"; Table tbl = createTable(tblName, SCHEMA, PartitionSpec.unpartitioned()); From 9923ac9384900f9644efa4118ef953095f15d0fd Mon Sep 17 00:00:00 2001 From: Sai Tharun <56219964+saitharun15@users.noreply.github.com> Date: Thu, 14 Nov 2024 14:57:19 +0530 Subject: [PATCH 137/313] Spark 3.4: Support Spark Column Stats (#11532) --- .../apache/iceberg/spark/SparkReadConf.java | 8 + .../iceberg/spark/SparkSQLProperties.java | 4 + .../spark/source/SparkChangelogScan.java | 2 +- .../spark/source/SparkColumnStatistics.java | 88 +++++ .../iceberg/spark/source/SparkScan.java | 60 +++- .../apache/iceberg/spark/source/Stats.java | 12 +- .../iceberg/spark/source/TestSparkScan.java | 305 ++++++++++++++++++ .../iceberg/spark/source/TestSparkScan.java | 2 +- 8 files changed, 475 insertions(+), 6 deletions(-) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index f0c6b9115c7b..fdc9347bc3d1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -351,4 +351,12 @@ private boolean executorCacheLocalityEnabledInternal() { .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT) .parse(); } + + public boolean reportColumnStats() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.REPORT_COLUMN_STATS) + .defaultValue(SparkSQLProperties.REPORT_COLUMN_STATS_DEFAULT) + .parse(); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index fde8dd29f02a..1e8c732d2d33 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -97,4 +97,8 @@ private SparkSQLProperties() {} public static final String EXECUTOR_CACHE_LOCALITY_ENABLED = "spark.sql.iceberg.executor-cache.locality.enabled"; public static final boolean EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT = false; + + // Controls whether to report available column statistics to Spark for query optimization. + public static final String REPORT_COLUMN_STATS = "spark.sql.iceberg.report-column-stats"; + public static final boolean REPORT_COLUMN_STATS_DEFAULT = true; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java index 7cde3e1fbe11..71b53d70262f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java @@ -88,7 +88,7 @@ class SparkChangelogScan implements Scan, SupportsReportStatistics { public Statistics estimateStatistics() { long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum(); long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount); - return new Stats(sizeInBytes, rowsCount); + return new Stats(sizeInBytes, rowsCount, Collections.emptyMap()); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java new file mode 100644 index 000000000000..faaff3631d7c --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.util.Optional; +import java.util.OptionalLong; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; +import org.apache.spark.sql.connector.read.colstats.Histogram; + +class SparkColumnStatistics implements ColumnStatistics { + + private final OptionalLong distinctCount; + private final Optional min; + private final Optional max; + private final OptionalLong nullCount; + private final OptionalLong avgLen; + private final OptionalLong maxLen; + private final Optional histogram; + + SparkColumnStatistics( + Long distinctCount, + Object min, + Object max, + Long nullCount, + Long avgLen, + Long maxLen, + Histogram histogram) { + this.distinctCount = + (distinctCount == null) ? OptionalLong.empty() : OptionalLong.of(distinctCount); + this.min = Optional.ofNullable(min); + this.max = Optional.ofNullable(max); + this.nullCount = (nullCount == null) ? OptionalLong.empty() : OptionalLong.of(nullCount); + this.avgLen = (avgLen == null) ? OptionalLong.empty() : OptionalLong.of(avgLen); + this.maxLen = (maxLen == null) ? OptionalLong.empty() : OptionalLong.of(maxLen); + this.histogram = Optional.ofNullable(histogram); + } + + @Override + public OptionalLong distinctCount() { + return distinctCount; + } + + @Override + public Optional min() { + return min; + } + + @Override + public Optional max() { + return max; + } + + @Override + public OptionalLong nullCount() { + return nullCount; + } + + @Override + public OptionalLong avgLen() { + return avgLen; + } + + @Override + public OptionalLong maxLen() { + return maxLen; + } + + @Override + public Optional histogram() { + return histogram; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 6efe8a080bde..019f3919dc57 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -23,15 +23,19 @@ import java.util.Map; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.iceberg.BlobMetadata; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -75,22 +79,28 @@ import org.apache.iceberg.util.TableScanUtil; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.expressions.FieldReference; +import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.connector.metric.CustomMetric; import org.apache.spark.sql.connector.metric.CustomTaskMetric; import org.apache.spark.sql.connector.read.Batch; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsReportStatistics; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; abstract class SparkScan implements Scan, SupportsReportStatistics { private static final Logger LOG = LoggerFactory.getLogger(SparkScan.class); + private static final String NDV_KEY = "ndv"; private final JavaSparkContext sparkContext; private final Table table; + private final SparkSession spark; private final SparkReadConf readConf; private final boolean caseSensitive; private final Schema expectedSchema; @@ -111,6 +121,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch()); SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema); + this.spark = spark; this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); this.table = table; this.readConf = readConf; @@ -175,7 +186,49 @@ public Statistics estimateStatistics() { protected Statistics estimateStatistics(Snapshot snapshot) { // its a fresh table, no data if (snapshot == null) { - return new Stats(0L, 0L); + return new Stats(0L, 0L, Collections.emptyMap()); + } + + boolean cboEnabled = + Boolean.parseBoolean(spark.conf().get(SQLConf.CBO_ENABLED().key(), "false")); + Map colStatsMap = Collections.emptyMap(); + if (readConf.reportColumnStats() && cboEnabled) { + colStatsMap = Maps.newHashMap(); + List files = table.statisticsFiles(); + if (!files.isEmpty()) { + List metadataList = (files.get(0)).blobMetadata(); + + Map> groupedByField = + metadataList.stream() + .collect( + Collectors.groupingBy( + metadata -> metadata.fields().get(0), Collectors.toList())); + + for (Map.Entry> entry : groupedByField.entrySet()) { + String colName = table.schema().findColumnName(entry.getKey()); + NamedReference ref = FieldReference.column(colName); + Long ndv = null; + + for (BlobMetadata blobMetadata : entry.getValue()) { + if (blobMetadata + .type() + .equals(org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1)) { + String ndvStr = blobMetadata.properties().get(NDV_KEY); + if (!Strings.isNullOrEmpty(ndvStr)) { + ndv = Long.parseLong(ndvStr); + } else { + LOG.debug("{} is not set in BlobMetadata for column {}", NDV_KEY, colName); + } + } else { + LOG.debug("Blob type {} is not supported yet", blobMetadata.type()); + } + } + ColumnStatistics colStats = + new SparkColumnStatistics(ndv, null, null, null, null, null, null); + + colStatsMap.put(ref, colStats); + } + } } // estimate stats using snapshot summary only for partitioned tables @@ -186,12 +239,13 @@ protected Statistics estimateStatistics(Snapshot snapshot) { snapshot.snapshotId(), table.name()); long totalRecords = totalRecords(snapshot); - return new Stats(SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords); + return new Stats( + SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords, colStatsMap); } long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum(); long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount); - return new Stats(sizeInBytes, rowsCount); + return new Stats(sizeInBytes, rowsCount, colStatsMap); } private long totalRecords(Snapshot snapshot) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java index ddf6ca834d9b..ccf523cb4b05 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java @@ -18,16 +18,21 @@ */ package org.apache.iceberg.spark.source; +import java.util.Map; import java.util.OptionalLong; +import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; class Stats implements Statistics { private final OptionalLong sizeInBytes; private final OptionalLong numRows; + private final Map colstats; - Stats(long sizeInBytes, long numRows) { + Stats(long sizeInBytes, long numRows, Map colstats) { this.sizeInBytes = OptionalLong.of(sizeInBytes); this.numRows = OptionalLong.of(numRows); + this.colstats = colstats; } @Override @@ -39,4 +44,9 @@ public OptionalLong sizeInBytes() { public OptionalLong numRows() { return numRows; } + + @Override + public Map columnStats() { + return colstats; + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index bc03fa042920..45f5e925ca6a 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.source; +import static org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createPartitionedTable; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createUnpartitionedTable; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToDayOrdinal; @@ -28,10 +29,18 @@ import static org.apache.spark.sql.functions.expr; import static org.assertj.core.api.Assertions.assertThat; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; 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.Maps; import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.iceberg.spark.functions.BucketFunction; import org.apache.iceberg.spark.functions.DaysFunction; @@ -40,6 +49,7 @@ import org.apache.iceberg.spark.functions.TruncateFunction; import org.apache.iceberg.spark.functions.YearsFunction; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; @@ -56,6 +66,8 @@ import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.junit.After; @@ -68,6 +80,8 @@ @RunWith(Parameterized.class) public class TestSparkScan extends SparkTestBaseWithCatalog { + private static final String DUMMY_BLOB_TYPE = "sum-data-size-bytes-v1"; + private final String format; @Parameterized.Parameters(name = "format = {0}") @@ -112,6 +126,271 @@ public void testEstimatedRowCount() throws NoSuchTableException { Assert.assertEquals(10000L, stats.numRows().getAsLong()); } + @Test + public void testTableWithoutColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + // The expected col NDVs are nulls + withSQLConf( + reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); + } + + @Test + public void testTableWithoutApacheDatasketchColStat() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + DUMMY_BLOB_TYPE, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("data_size", "4")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + // The expected col NDVs are nulls + withSQLConf( + reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); + } + + @Test + public void testTableWithOneColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedOneNDV = Maps.newHashMap(); + expectedOneNDV.put("id", 4L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedOneNDV)); + } + + @Test + public void testTableWithOneApacheDatasketchColStatAndOneDifferentColStat() + throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")), + new GenericBlobMetadata( + DUMMY_BLOB_TYPE, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("data_size", "2")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedOneNDV = Maps.newHashMap(); + expectedOneNDV.put("id", 4L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedOneNDV)); + } + + @Test + public void testTableWithTwoColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")), + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(2), + ImmutableMap.of("ndv", "2")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedTwoNDVs = Maps.newHashMap(); + expectedTwoNDVs.put("id", 4L); + expectedTwoNDVs.put("data", 2L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedTwoNDVs)); + } + @Test public void testUnpartitionedYears() throws Exception { createUnpartitionedTable(spark, tableName); @@ -716,6 +995,32 @@ private Expression[] expressions(Expression... expressions) { return expressions; } + private void checkColStatisticsNotReported(SparkScan scan, long expectedRowCount) { + Statistics stats = scan.estimateStatistics(); + assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); + + Map columnStats = stats.columnStats(); + assertThat(columnStats).isEmpty(); + } + + private void checkColStatisticsReported( + SparkScan scan, long expectedRowCount, Map expectedNDVs) { + Statistics stats = scan.estimateStatistics(); + assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); + + Map columnStats = stats.columnStats(); + if (expectedNDVs.isEmpty()) { + assertThat(columnStats.values().stream().allMatch(value -> value.distinctCount().isEmpty())) + .isTrue(); + } else { + for (Map.Entry entry : expectedNDVs.entrySet()) { + assertThat( + columnStats.get(FieldReference.column(entry.getKey())).distinctCount().getAsLong()) + .isEqualTo(entry.getValue()); + } + } + } + private static LiteralValue intLit(int value) { return LiteralValue.apply(value, DataTypes.IntegerType); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index af144fe4bf3d..dbb15ca5a743 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -1018,7 +1018,7 @@ private void checkColStatisticsNotReported(SparkScan scan, long expectedRowCount assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); Map columnStats = stats.columnStats(); - assertThat(columnStats.isEmpty()); + assertThat(columnStats).isEmpty(); } private void checkColStatisticsReported( From daa24f9c3a56e18d188097deb2dd79cc991c9a78 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 14 Nov 2024 18:48:48 +0800 Subject: [PATCH 138/313] Docs: Fix rendering lists (#11546) --- format/puffin-spec.md | 1 + format/spec.md | 2 ++ format/view-spec.md | 2 ++ 3 files changed, 5 insertions(+) diff --git a/format/puffin-spec.md b/format/puffin-spec.md index 0148db72e276..62e8ae085398 100644 --- a/format/puffin-spec.md +++ b/format/puffin-spec.md @@ -145,6 +145,7 @@ sub-position) are tested for inclusion in the bitmap. If a bitmap is not found for the key, then it is not set. The serialized blob contains: + * Combined length of the vector and magic bytes stored as 4 bytes, big-endian * A 4-byte magic sequence, `D1 D3 39 64` * The vector, serialized as described below diff --git a/format/spec.md b/format/spec.md index c9a5f715102d..631008c5daf1 100644 --- a/format/spec.md +++ b/format/spec.md @@ -262,6 +262,7 @@ Iceberg's Avro manifest format does not store the type of lower and upper bounds | `decimal(P, S)` | _any_ | `decimal(P', S)`; `P' <= P` | Type promotion is not allowed for a field that is referenced by `source-id` or `source-ids` of a partition field if the partition transform would produce a different value after promoting the type. For example, `bucket[N]` produces different hash values for `34` and `"34"` (2017239379 != -427558391) but the same value for `34` and `34L`; when an `int` field is the source for a bucket partition field, it may be promoted to `long` but not to `string`. This may happen for the following type promotion cases: + * `date` to `timestamp` or `timestamp_ns` Any struct, including a top-level schema, can evolve through deleting fields, adding new fields, renaming existing fields, reordering existing fields, or promoting a primitive using the valid type promotions. Adding a new field assigns a new ID for that field and for any nested fields. Renaming an existing field must change the name, but not the field ID. Deleting a field removes it from the current schema. Field deletion cannot be rolled back unless the field was nullable or if the current snapshot has not changed. @@ -1001,6 +1002,7 @@ Notes: This section details how to encode row-level deletes in Iceberg delete files. Row-level deletes are added by v2 and are not supported in v1. Deletion vectors are added in v3 and are not supported in v2 or earlier. Position delete files must not be added to v3 tables, but existing position delete files are valid. There are three types of row-level deletes: + * Deletion vectors (DVs) identify deleted rows within a single referenced data file by position in a bitmap * Position delete files identify deleted rows by file location and row position (**deprecated**) * Equality delete files identify deleted rows by the value of one or more columns diff --git a/format/view-spec.md b/format/view-spec.md index ff44d9d3bb3d..011781d9e4ab 100644 --- a/format/view-spec.md +++ b/format/view-spec.md @@ -65,6 +65,7 @@ The view version metadata file has the following fields: | _optional_ | `properties` | A string to string map of view properties [2] | Notes: + 1. The number of versions to retain is controlled by the table property: `version.history.num-entries`. 2. Properties are used for metadata such as `comment` and for settings that affect view maintenance. This is not intended to be used for arbitrary metadata. @@ -102,6 +103,7 @@ A view version can have more than one representation. All representations for a View versions are immutable. Once a version is created, it cannot be changed. This means that representations for a version cannot be changed. If a view definition changes (or new representations are to be added), a new version must be created. Each representation is an object with at least one common field, `type`, that is one of the following: + * `sql`: a SQL SELECT statement that defines the view Representations further define metadata for each type. From 071d9e277833cfb13421be345c67f4a1876b4840 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Nov 2024 06:35:32 -0800 Subject: [PATCH 139/313] Build: Bump kafka from 3.8.1 to 3.9.0 (#11508) Bumps `kafka` from 3.8.1 to 3.9.0. Updates `org.apache.kafka:kafka-clients` from 3.8.1 to 3.9.0 Updates `org.apache.kafka:connect-api` from 3.8.1 to 3.9.0 Updates `org.apache.kafka:connect-json` from 3.8.1 to 3.9.0 --- updated-dependencies: - dependency-name: org.apache.kafka:kafka-clients dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.apache.kafka:connect-api dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.apache.kafka:connect-json dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 57d1810bd23c..54a9f14b4b42 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -65,7 +65,7 @@ jaxb-runtime = "2.3.9" jetty = "11.0.24" junit = "5.11.3" junit-platform = "1.11.3" -kafka = "3.8.1" +kafka = "3.9.0" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" mockito = "4.11.0" From 09634857e4a1333f5dc742d1dca3921e9a9f62dd Mon Sep 17 00:00:00 2001 From: Marc Cenac <547446+mrcnc@users.noreply.github.com> Date: Thu, 14 Nov 2024 08:38:04 -0600 Subject: [PATCH 140/313] Support WASB scheme in ADLSFileIO (#11504) --- .../apache/iceberg/azure/AzureProperties.java | 13 +++++++++++- .../iceberg/azure/adlsv2/ADLSLocation.java | 20 +++++++++++++------ .../iceberg/azure/AzurePropertiesTest.java | 10 ++++++---- .../azure/adlsv2/ADLSLocationTest.java | 19 ++++++++++++++---- .../apache/iceberg/io/ResolvingFileIO.java | 4 +++- 5 files changed, 50 insertions(+), 16 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index 2d363cbc5231..a7f9885a4726 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -77,6 +77,17 @@ public Optional adlsWriteBlockSize() { return Optional.ofNullable(adlsWriteBlockSize); } + /** + * Applies configuration to the {@link DataLakeFileSystemClientBuilder} to provide the endpoint + * and credentials required to create an instance of the client. + * + *

    The default endpoint is constructed in the form {@code + * https://{account}.dfs.core.windows.net} and default credentials are provided via the {@link + * com.azure.identity.DefaultAzureCredential}. + * + * @param account the service account name + * @param builder the builder instance + */ public void applyClientConfiguration(String account, DataLakeFileSystemClientBuilder builder) { String sasToken = adlsSasTokens.get(account); if (sasToken != null && !sasToken.isEmpty()) { @@ -93,7 +104,7 @@ public void applyClientConfiguration(String account, DataLakeFileSystemClientBui if (connectionString != null && !connectionString.isEmpty()) { builder.endpoint(connectionString); } else { - builder.endpoint("https://" + account); + builder.endpoint("https://" + account + ".dfs.core.windows.net"); } } } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java index 5af590628fe8..fb91c4cb3233 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java @@ -30,14 +30,21 @@ * *

    Locations follow a URI like structure to identify resources * - *

    {@code abfs[s]://[@]/}
    + *
    {@code abfs[s]://[@].dfs.core.windows.net/}
    + * + * or + * + *
    {@code wasb[s]://@.blob.core.windows.net/}
    + * + * For compatibility, locations using the wasb scheme are also accepted but will use the Azure Data + * Lake Storage Gen2 REST APIs instead of the Blob Storage REST APIs. * *

    See Azure * Data Lake Storage URI */ class ADLSLocation { - private static final Pattern URI_PATTERN = Pattern.compile("^abfss?://([^/?#]+)(.*)?$"); + private static final Pattern URI_PATTERN = Pattern.compile("^(abfss?|wasbs?)://([^/?#]+)(.*)?$"); private final String storageAccount; private final String container; @@ -55,17 +62,18 @@ class ADLSLocation { ValidationException.check(matcher.matches(), "Invalid ADLS URI: %s", location); - String authority = matcher.group(1); + String authority = matcher.group(2); String[] parts = authority.split("@", -1); if (parts.length > 1) { this.container = parts[0]; - this.storageAccount = parts[1]; + String host = parts[1]; + this.storageAccount = host.split("\\.", -1)[0]; } else { this.container = null; - this.storageAccount = authority; + this.storageAccount = authority.split("\\.", -1)[0]; } - String uriPath = matcher.group(2); + String uriPath = matcher.group(3); this.path = uriPath == null ? "" : uriPath.startsWith("/") ? uriPath.substring(1) : uriPath; } diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index 6b8287c44e58..4f032d7ab125 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -97,11 +97,13 @@ public void testNoSasToken() { @Test public void testWithConnectionString() { AzureProperties props = - new AzureProperties(ImmutableMap.of("adls.connection-string.account1", "http://endpoint")); + new AzureProperties( + ImmutableMap.of( + "adls.connection-string.account1", "https://account1.dfs.core.usgovcloudapi.net")); DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account1", clientBuilder); - verify(clientBuilder).endpoint("http://endpoint"); + verify(clientBuilder).endpoint("https://account1.dfs.core.usgovcloudapi.net"); } @Test @@ -111,7 +113,7 @@ public void testNoMatchingConnectionString() { DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account1", clientBuilder); - verify(clientBuilder).endpoint("https://account1"); + verify(clientBuilder).endpoint("https://account1.dfs.core.windows.net"); } @Test @@ -120,7 +122,7 @@ public void testNoConnectionString() { DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account", clientBuilder); - verify(clientBuilder).endpoint("https://account"); + verify(clientBuilder).endpoint("https://account.dfs.core.windows.net"); } @Test diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java index 403886f4b28e..10b5e1877cca 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java @@ -33,7 +33,18 @@ public void testLocationParsing(String scheme) { String p1 = scheme + "://container@account.dfs.core.windows.net/path/to/file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + assertThat(location.storageAccount()).isEqualTo("account"); + assertThat(location.container().get()).isEqualTo("container"); + assertThat(location.path()).isEqualTo("path/to/file"); + } + + @ParameterizedTest + @ValueSource(strings = {"wasb", "wasbs"}) + public void testWasbLocatonParsing(String scheme) { + String p1 = scheme + "://container@account.blob.core.windows.net/path/to/file"; + ADLSLocation location = new ADLSLocation(p1); + + assertThat(location.storageAccount()).isEqualTo("account"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path/to/file"); } @@ -43,7 +54,7 @@ public void testEncodedString() { String p1 = "abfs://container@account.dfs.core.windows.net/path%20to%20file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + assertThat(location.storageAccount()).isEqualTo("account"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path%20to%20file"); } @@ -67,7 +78,7 @@ public void testNoContainer() { String p1 = "abfs://account.dfs.core.windows.net/path/to/file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + assertThat(location.storageAccount()).isEqualTo("account"); assertThat(location.container().isPresent()).isFalse(); assertThat(location.path()).isEqualTo("path/to/file"); } @@ -77,7 +88,7 @@ public void testNoPath() { String p1 = "abfs://container@account.dfs.core.windows.net"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); + assertThat(location.storageAccount()).isEqualTo("account"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo(""); } diff --git a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java index a858045aab8b..a8adf979f85a 100644 --- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java +++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java @@ -62,7 +62,9 @@ public class ResolvingFileIO implements HadoopConfigurable, DelegateFileIO { "s3n", S3_FILE_IO_IMPL, "gs", GCS_FILE_IO_IMPL, "abfs", ADLS_FILE_IO_IMPL, - "abfss", ADLS_FILE_IO_IMPL); + "abfss", ADLS_FILE_IO_IMPL, + "wasb", ADLS_FILE_IO_IMPL, + "wasbs", ADLS_FILE_IO_IMPL); private final Map ioInstances = Maps.newConcurrentMap(); private final AtomicBoolean isClosed = new AtomicBoolean(false); From 0a705b0637db484730eb4eece69ae6c4d52fd9da Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 14 Nov 2024 16:04:01 -0600 Subject: [PATCH 141/313] Docs: 4 Spaces are Requried for Sublists (#11549) --- site/docs/releases.md | 130 +++++++++++++++++++++--------------------- 1 file changed, 65 insertions(+), 65 deletions(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index e90887282514..5346d2a476d8 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -77,79 +77,79 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: The 1.7.0 release contains fixes, dependency updates, and new features. For full release notes please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.7.0). An abridged list follows * Deprecation / End of Support - - Java 8 - - Apache Pig + - Java 8 + - Apache Pig * API - - Add SupportsRecoveryOperations mixin for FileIO ([\#10711](https://github.com/apache/iceberg/pull/10711)) - - Add default value APIs and Avro implementation ([\#9502](https://github.com/apache/iceberg/pull/9502)) - - Add compatibility checks for Schemas with default values ([\#11434](https://github.com/apache/iceberg/pull/11434)) - - Implement types timestamp_ns and timestamptz_ns ([\#9008](https://github.com/apache/iceberg/pull/9008)) - - Add addNonDefaultSpec to UpdatePartitionSpec to not set the new partition spec as default ([\#10736](https://github.com/apache/iceberg/pull/10736)) + - Add SupportsRecoveryOperations mixin for FileIO ([\#10711](https://github.com/apache/iceberg/pull/10711)) + - Add default value APIs and Avro implementation ([\#9502](https://github.com/apache/iceberg/pull/9502)) + - Add compatibility checks for Schemas with default values ([\#11434](https://github.com/apache/iceberg/pull/11434)) + - Implement types timestamp_ns and timestamptz_ns ([\#9008](https://github.com/apache/iceberg/pull/9008)) + - Add addNonDefaultSpec to UpdatePartitionSpec to not set the new partition spec as default ([\#10736](https://github.com/apache/iceberg/pull/10736)) * AWS - - Don't complete multipart upload on finalize for S3OutputStream ([\#10874](https://github.com/apache/iceberg/pull/10874)) - - Implement SupportsRecoveryOperations for S3FileIO ([\#10721](https://github.com/apache/iceberg/pull/10721)) - - Refresh vended credentials ([\#11389](https://github.com/apache/iceberg/pull/11389)) - - Support S3 directory bucket listing ([\#11021](https://github.com/apache/iceberg/pull/11021)) - - S3FileIO - Add Cross-Region Bucket Access ([\#11259](https://github.com/apache/iceberg/pull/11259)) + - Don't complete multipart upload on finalize for S3OutputStream ([\#10874](https://github.com/apache/iceberg/pull/10874)) + - Implement SupportsRecoveryOperations for S3FileIO ([\#10721](https://github.com/apache/iceberg/pull/10721)) + - Refresh vended credentials ([\#11389](https://github.com/apache/iceberg/pull/11389)) + - Support S3 directory bucket listing ([\#11021](https://github.com/apache/iceberg/pull/11021)) + - S3FileIO - Add Cross-Region Bucket Access ([\#11259](https://github.com/apache/iceberg/pull/11259)) * Build - - Build for Java 11 ([\#10849](https://github.com/apache/iceberg/pull/10849)) - Removal of Java 8 - - Build: Add checkstyle rule to ban assert usage ([\#10886](https://github.com/apache/iceberg/pull/10886)) - - Support building with Java 21 ([\#10474](https://github.com/apache/iceberg/pull/10474)) + - Build for Java 11 ([\#10849](https://github.com/apache/iceberg/pull/10849)) - Removal of Java 8 + - Build: Add checkstyle rule to ban assert usage ([\#10886](https://github.com/apache/iceberg/pull/10886)) + - Support building with Java 21 ([\#10474](https://github.com/apache/iceberg/pull/10474)) * Dependencies - - AWS SDK 2.29.1 - - Apache Avro to 1.12.0 - - Spark 3.4 to 3.4.4 - - Spark 3.5 to 3.5.2 - - Netty to 4.1.114.Final - - Jetty to 11.0.24 - - Kafka to 3.8.0 - - Nessie to 0.99.0 - - ORC to 1.9.4 - - Roaring Bitmap to 1.3.0 - - Spring to 5.3.39 - - Sqllite JDBC to 3.46.0.0 - - Hadoop to 3.4.1 + - AWS SDK 2.29.1 + - Apache Avro to 1.12.0 + - Spark 3.4 to 3.4.4 + - Spark 3.5 to 3.5.2 + - Netty to 4.1.114.Final + - Jetty to 11.0.24 + - Kafka to 3.8.0 + - Nessie to 0.99.0 + - ORC to 1.9.4 + - Roaring Bitmap to 1.3.0 + - Spring to 5.3.39 + - Sqllite JDBC to 3.46.0.0 + - Hadoop to 3.4.1 * Core - - Remove dangling deletes as part of RewriteDataFilesAction ([\#9724](https://github.com/apache/iceberg/pull/9724)) - - Add a util to compute partition stats ([\#11146](https://github.com/apache/iceberg/pull/11146)) - - Add estimateRowCount for Files and Entries Metadata Tables ([\#10759](https://github.com/apache/iceberg/pull/10759)) - - Add portable Roaring bitmap for row positions ([\#11372](https://github.com/apache/iceberg/pull/11372)) - - Add rewritten delete files to write results ([\#11203](https://github.com/apache/iceberg/pull/11203)) - - Add Basic Classes for Iceberg Table Version 3 ([\#10760](https://github.com/apache/iceberg/pull/10760)) - - Deprecate ContentCache.invalidateAll ([\#10494](https://github.com/apache/iceberg/pull/10494)) - - Deprecate legacy ways for loading position deletes ([\#11242](https://github.com/apache/iceberg/pull/11242)) - - Parallelize manifest writing for many new files ([\#11086](https://github.com/apache/iceberg/pull/11086)) - - Support appending files with different specs ([\#9860](https://github.com/apache/iceberg/pull/9860)) + - Remove dangling deletes as part of RewriteDataFilesAction ([\#9724](https://github.com/apache/iceberg/pull/9724)) + - Add a util to compute partition stats ([\#11146](https://github.com/apache/iceberg/pull/11146)) + - Add estimateRowCount for Files and Entries Metadata Tables ([\#10759](https://github.com/apache/iceberg/pull/10759)) + - Add portable Roaring bitmap for row positions ([\#11372](https://github.com/apache/iceberg/pull/11372)) + - Add rewritten delete files to write results ([\#11203](https://github.com/apache/iceberg/pull/11203)) + - Add Basic Classes for Iceberg Table Version 3 ([\#10760](https://github.com/apache/iceberg/pull/10760)) + - Deprecate ContentCache.invalidateAll ([\#10494](https://github.com/apache/iceberg/pull/10494)) + - Deprecate legacy ways for loading position deletes ([\#11242](https://github.com/apache/iceberg/pull/11242)) + - Parallelize manifest writing for many new files ([\#11086](https://github.com/apache/iceberg/pull/11086)) + - Support appending files with different specs ([\#9860](https://github.com/apache/iceberg/pull/9860)) * Flink - - Introduces the new IcebergSink based on the new V2 Flink Sink Abstraction ([\#10179](https://github.com/apache/iceberg/pull/10179)) - - Update Flink to use planned Avro reads ([\#11386](https://github.com/apache/iceberg/pull/11386)) - - Infer source parallelism for FLIP-27 source in batch execution mode ([\#10832](https://github.com/apache/iceberg/pull/10832)) - - Make FLIP-27 default in SQL and mark the old FlinkSource as deprecated ([\#11345](https://github.com/apache/iceberg/pull/11345)) - - Support limit pushdown in FLIP-27 source ([\#10748](https://github.com/apache/iceberg/pull/10748)) + - Introduces the new IcebergSink based on the new V2 Flink Sink Abstraction ([\#10179](https://github.com/apache/iceberg/pull/10179)) + - Update Flink to use planned Avro reads ([\#11386](https://github.com/apache/iceberg/pull/11386)) + - Infer source parallelism for FLIP-27 source in batch execution mode ([\#10832](https://github.com/apache/iceberg/pull/10832)) + - Make FLIP-27 default in SQL and mark the old FlinkSource as deprecated ([\#11345](https://github.com/apache/iceberg/pull/11345)) + - Support limit pushdown in FLIP-27 source ([\#10748](https://github.com/apache/iceberg/pull/10748)) * GCS - - Refresh vended credentials ([\#11282](https://github.com/apache/iceberg/pull/11282)) + - Refresh vended credentials ([\#11282](https://github.com/apache/iceberg/pull/11282)) * Hive - - Add View support for HIVE catalog ([\#9852](https://github.com/apache/iceberg/pull/9852)) + - Add View support for HIVE catalog ([\#9852](https://github.com/apache/iceberg/pull/9852)) * OpenAPI - - Add RemovePartitionSpecsUpdate REST update type ([\#10846](https://github.com/apache/iceberg/pull/10846)) - - Add endpoint to retrieve valid credentials for a given table ([\#11281](https://github.com/apache/iceberg/pull/11281)) - - Standardize credentials in loadTable/loadView responses ([\#10722](https://github.com/apache/iceberg/pull/10722)) - - Add Scan Planning Endpoints to open api spec ([\#9695](https://github.com/apache/iceberg/pull/9695)) - - Add REST Compatibility Kit ([\#10908](https://github.com/apache/iceberg/pull/10908)) + - Add RemovePartitionSpecsUpdate REST update type ([\#10846](https://github.com/apache/iceberg/pull/10846)) + - Add endpoint to retrieve valid credentials for a given table ([\#11281](https://github.com/apache/iceberg/pull/11281)) + - Standardize credentials in loadTable/loadView responses ([\#10722](https://github.com/apache/iceberg/pull/10722)) + - Add Scan Planning Endpoints to open api spec ([\#9695](https://github.com/apache/iceberg/pull/9695)) + - Add REST Compatibility Kit ([\#10908](https://github.com/apache/iceberg/pull/10908)) * Spark - - Parallelize reading files in migrate procedures ([\#11043](https://github.com/apache/iceberg/pull/11043)) - - Action to compute table stats ([\#11106](https://github.com/apache/iceberg/pull/11106)) - - Action to remove dangling deletes ([\#11377](https://github.com/apache/iceberg/pull/11377)) - - Add utility to load table state reliably ([\#11115](https://github.com/apache/iceberg/pull/11115)) - - Don't change table distribution when only altering local order ([\#10774](https://github.com/apache/iceberg/pull/10774)) - - Update Spark to use planned Avro reads ([\#11299](https://github.com/apache/iceberg/pull/11299)) - - Spark Action to Analyze table ([\#10288](https://github.com/apache/iceberg/pull/10288)) - - Support Column Stats ([\#10659](https://github.com/apache/iceberg/pull/10659)) - - Add RewriteTablePath action interface ([\#10920](https://github.com/apache/iceberg/pull/10920)) + - Parallelize reading files in migrate procedures ([\#11043](https://github.com/apache/iceberg/pull/11043)) + - Action to compute table stats ([\#11106](https://github.com/apache/iceberg/pull/11106)) + - Action to remove dangling deletes ([\#11377](https://github.com/apache/iceberg/pull/11377)) + - Add utility to load table state reliably ([\#11115](https://github.com/apache/iceberg/pull/11115)) + - Don't change table distribution when only altering local order ([\#10774](https://github.com/apache/iceberg/pull/10774)) + - Update Spark to use planned Avro reads ([\#11299](https://github.com/apache/iceberg/pull/11299)) + - Spark Action to Analyze table ([\#10288](https://github.com/apache/iceberg/pull/10288)) + - Support Column Stats ([\#10659](https://github.com/apache/iceberg/pull/10659)) + - Add RewriteTablePath action interface ([\#10920](https://github.com/apache/iceberg/pull/10920)) * Spec - - Add v3 types and type promotion ([\#10955](https://github.com/apache/iceberg/pull/10955)) - - Adds Row Lineage ([\#11130](https://github.com/apache/iceberg/pull/11130)) - - Deprecate the file system table scheme. ([\#10833](https://github.com/apache/iceberg/pull/10833)) + - Add v3 types and type promotion ([\#10955](https://github.com/apache/iceberg/pull/10955)) + - Adds Row Lineage ([\#11130](https://github.com/apache/iceberg/pull/11130)) + - Deprecate the file system table scheme. ([\#10833](https://github.com/apache/iceberg/pull/10833)) ### 1.6.1 release @@ -158,10 +158,10 @@ Apache Iceberg 1.6.1 was released on August 27, 2024. The 1.6.1 Release contains bug fixes and performance improvements. For full release notes visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.6.1) * Core - - Limit ParallelIterable memory consumption by yielding in tasks ([\#10787](https://github.com/apache/iceberg/#10787)) - - Drop ParallelIterable's queue low water mark ([\#10979](https://github.com/apache/iceberg/#10979)) + - Limit ParallelIterable memory consumption by yielding in tasks ([\#10787](https://github.com/apache/iceberg/#10787)) + - Drop ParallelIterable's queue low water mark ([\#10979](https://github.com/apache/iceberg/#10979)) * Dependencies - - ORC 1.9.4 + - ORC 1.9.4 ### 1.6.0 release From 4c0288a4ca86d0cf400405777808e2eb706d8d25 Mon Sep 17 00:00:00 2001 From: Rocco Varela Date: Thu, 14 Nov 2024 22:07:04 -0800 Subject: [PATCH 142/313] API, Core, Spark: Ignore schema merge updates from long -> int (#11419) --- .../java/org/apache/iceberg/UpdateSchema.java | 4 +- .../iceberg/schema/UnionByNameVisitor.java | 21 ++- .../iceberg/TestSchemaUnionByFieldName.java | 28 +++- .../spark/source/TestDataFrameWriterV2.java | 135 ++++++++++++++++++ .../spark/source/TestDataFrameWriterV2.java | 135 ++++++++++++++++++ .../spark/source/TestDataFrameWriterV2.java | 134 +++++++++++++++++ 6 files changed, 450 insertions(+), 7 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/UpdateSchema.java b/api/src/main/java/org/apache/iceberg/UpdateSchema.java index afe1891cd530..c84c237f8d8f 100644 --- a/api/src/main/java/org/apache/iceberg/UpdateSchema.java +++ b/api/src/main/java/org/apache/iceberg/UpdateSchema.java @@ -369,7 +369,9 @@ default UpdateSchema updateColumn(String name, Type.PrimitiveType newType, Strin * to create a union schema. * *

    For fields with same canonical names in both schemas it is required that the widen types is - * supported using {@link UpdateSchema#updateColumn(String, Type.PrimitiveType)} + * supported using {@link UpdateSchema#updateColumn(String, Type.PrimitiveType)}. Differences in + * type are ignored if the new type is narrower than the existing type (e.g. long to int, double + * to float). * *

    Only supports turning a previously required field into an optional one if it is marked * optional in the provided new schema using {@link UpdateSchema#makeColumnOptional(String)} diff --git a/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java b/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java index 1497ba59c582..68172b7062a6 100644 --- a/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java +++ b/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java @@ -24,6 +24,7 @@ import org.apache.iceberg.UpdateSchema; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; /** @@ -163,8 +164,7 @@ private void updateColumn(Types.NestedField field, Types.NestedField existingFie String fullName = partnerSchema.findColumnName(existingField.fieldId()); boolean needsOptionalUpdate = field.isOptional() && existingField.isRequired(); - boolean needsTypeUpdate = - field.type().isPrimitiveType() && !field.type().equals(existingField.type()); + boolean needsTypeUpdate = !isIgnorableTypeUpdate(existingField.type(), field.type()); boolean needsDocUpdate = field.doc() != null && !field.doc().equals(existingField.doc()); if (needsOptionalUpdate) { @@ -180,6 +180,23 @@ private void updateColumn(Types.NestedField field, Types.NestedField existingFie } } + private boolean isIgnorableTypeUpdate(Type existingType, Type newType) { + if (existingType.isPrimitiveType()) { + // TypeUtil.isPromotionAllowed is used to check whether type promotion is allowed in the + // reverse order, newType to existingType. A true result implies that the newType is more + // narrow than the existingType, which translates in this context as an ignorable update when + // evaluating the existingType to newType order. A false result implies the opposite. + // Examples: + // existingType:long -> newType:int returns true, meaning it is ignorable + // existingType:int -> newType:long returns false, meaning it is not ignorable + return newType.isPrimitiveType() + && TypeUtil.isPromotionAllowed(newType, existingType.asPrimitiveType()); + } else { + // Complex -> Complex + return !newType.isPrimitiveType(); + } + } + private static class PartnerIdByNameAccessors implements PartnerAccessors { private final Schema partnerSchema; private boolean caseSensitive = true; diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java b/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java index bda76469e1fa..656e72a0c19c 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java @@ -303,13 +303,33 @@ public void testTypePromoteFloatToDouble() { } @Test - public void testInvalidTypePromoteDoubleToFloat() { + public void testIgnoreTypePromoteDoubleToFloat() { Schema currentSchema = new Schema(required(1, "aCol", DoubleType.get())); Schema newSchema = new Schema(required(1, "aCol", FloatType.get())); - assertThatThrownBy(() -> new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply()) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot change column type: aCol: double -> float"); + Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); + assertThat(applied.asStruct()).isEqualTo(currentSchema.asStruct()); + assertThat(applied.asStruct().fields()).hasSize(1); + assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(DoubleType.get()); + } + + @Test + public void testIgnoreTypePromoteLongToInt() { + Schema currentSchema = new Schema(required(1, "aCol", LongType.get())); + Schema newSchema = new Schema(required(1, "aCol", IntegerType.get())); + + Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); + assertThat(applied.asStruct().fields()).hasSize(1); + assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(LongType.get()); + } + + @Test + public void testIgnoreTypePromoteDecimalToNarrowerPrecision() { + Schema currentSchema = new Schema(required(1, "aCol", DecimalType.of(20, 1))); + Schema newSchema = new Schema(required(1, "aCol", DecimalType.of(10, 1))); + + Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); + assertThat(applied.asStruct()).isEqualTo(currentSchema.asStruct()); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java index 59a32bd239df..190f434e2949 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java @@ -18,13 +18,17 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.math.BigDecimal; import java.util.List; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; @@ -226,4 +230,135 @@ public void testWriteWithCaseSensitiveOption() throws NoSuchTableException, Pars fields = Spark3Util.loadIcebergTable(sparkSession, tableName).schema().asStruct().fields(); Assert.assertEquals(4, fields.size()); } + + @Test + public void testMergeSchemaIgnoreCastingLongToInt() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset bigintDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + bigintDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with long column", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + + Dataset intDF = + jsonToDF( + "id int, data string", + "{ \"id\": 3, \"data\": \"c\" }", + "{ \"id\": 4, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode(() -> intDF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged long column type", + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d")), + sql("select * from %s order by id", tableName)); + + // verify the column type did not change + Types.NestedField idField = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id"); + assertThat(idField.type().typeId()).isEqualTo(Type.TypeID.LONG); + } + + @Test + public void testMergeSchemaIgnoreCastingDoubleToFloat() throws Exception { + removeTables(); + sql("CREATE TABLE %s (id double, data string) USING iceberg", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset doubleDF = + jsonToDF( + "id double, data string", + "{ \"id\": 1.0, \"data\": \"a\" }", + "{ \"id\": 2.0, \"data\": \"b\" }"); + + doubleDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with double column", + ImmutableList.of(row(1.0, "a"), row(2.0, "b")), + sql("select * from %s order by id", tableName)); + + Dataset floatDF = + jsonToDF( + "id float, data string", + "{ \"id\": 3.0, \"data\": \"c\" }", + "{ \"id\": 4.0, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode(() -> floatDF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged double column type", + ImmutableList.of(row(1.0, "a"), row(2.0, "b"), row(3.0, "c"), row(4.0, "d")), + sql("select * from %s order by id", tableName)); + + // verify the column type did not change + Types.NestedField idField = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id"); + assertThat(idField.type().typeId()).isEqualTo(Type.TypeID.DOUBLE); + } + + @Test + public void testMergeSchemaIgnoreCastingDecimalToDecimalWithNarrowerPrecision() throws Exception { + removeTables(); + sql("CREATE TABLE %s (id decimal(6,2), data string) USING iceberg", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset decimalPrecision6DF = + jsonToDF( + "id decimal(6,2), data string", + "{ \"id\": 1.0, \"data\": \"a\" }", + "{ \"id\": 2.0, \"data\": \"b\" }"); + + decimalPrecision6DF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with decimal column with precision 6", + ImmutableList.of(row(new BigDecimal("1.00"), "a"), row(new BigDecimal("2.00"), "b")), + sql("select * from %s order by id", tableName)); + + Dataset decimalPrecision4DF = + jsonToDF( + "id decimal(4,2), data string", + "{ \"id\": 3.0, \"data\": \"c\" }", + "{ \"id\": 4.0, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode( + () -> decimalPrecision4DF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged decimal precision", + ImmutableList.of( + row(new BigDecimal("1.00"), "a"), + row(new BigDecimal("2.00"), "b"), + row(new BigDecimal("3.00"), "c"), + row(new BigDecimal("4.00"), "d")), + sql("select * from %s order by id", tableName)); + + // verify the decimal column precision did not change + Type idFieldType = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id").type(); + assertThat(idFieldType.typeId()).isEqualTo(Type.TypeID.DECIMAL); + Types.DecimalType decimalType = (Types.DecimalType) idFieldType; + assertThat(decimalType.precision()).isEqualTo(6); + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java index 824b0a17daef..47a0e87b9398 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java @@ -18,13 +18,17 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.math.BigDecimal; import java.util.List; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; @@ -209,4 +213,135 @@ public void testWriteWithCaseSensitiveOption() throws NoSuchTableException, Pars fields = Spark3Util.loadIcebergTable(sparkSession, tableName).schema().asStruct().fields(); Assert.assertEquals(4, fields.size()); } + + @Test + public void testMergeSchemaIgnoreCastingLongToInt() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset bigintDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + bigintDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with long column", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + + Dataset intDF = + jsonToDF( + "id int, data string", + "{ \"id\": 3, \"data\": \"c\" }", + "{ \"id\": 4, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode(() -> intDF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged long column type", + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d")), + sql("select * from %s order by id", tableName)); + + // verify the column type did not change + Types.NestedField idField = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id"); + assertThat(idField.type().typeId()).isEqualTo(Type.TypeID.LONG); + } + + @Test + public void testMergeSchemaIgnoreCastingDoubleToFloat() throws Exception { + removeTables(); + sql("CREATE TABLE %s (id double, data string) USING iceberg", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset doubleDF = + jsonToDF( + "id double, data string", + "{ \"id\": 1.0, \"data\": \"a\" }", + "{ \"id\": 2.0, \"data\": \"b\" }"); + + doubleDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with double column", + ImmutableList.of(row(1.0, "a"), row(2.0, "b")), + sql("select * from %s order by id", tableName)); + + Dataset floatDF = + jsonToDF( + "id float, data string", + "{ \"id\": 3.0, \"data\": \"c\" }", + "{ \"id\": 4.0, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode(() -> floatDF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged double column type", + ImmutableList.of(row(1.0, "a"), row(2.0, "b"), row(3.0, "c"), row(4.0, "d")), + sql("select * from %s order by id", tableName)); + + // verify the column type did not change + Types.NestedField idField = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id"); + assertThat(idField.type().typeId()).isEqualTo(Type.TypeID.DOUBLE); + } + + @Test + public void testMergeSchemaIgnoreCastingDecimalToDecimalWithNarrowerPrecision() throws Exception { + removeTables(); + sql("CREATE TABLE %s (id decimal(6,2), data string) USING iceberg", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset decimalPrecision6DF = + jsonToDF( + "id decimal(6,2), data string", + "{ \"id\": 1.0, \"data\": \"a\" }", + "{ \"id\": 2.0, \"data\": \"b\" }"); + + decimalPrecision6DF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with decimal column with precision 6", + ImmutableList.of(row(new BigDecimal("1.00"), "a"), row(new BigDecimal("2.00"), "b")), + sql("select * from %s order by id", tableName)); + + Dataset decimalPrecision4DF = + jsonToDF( + "id decimal(4,2), data string", + "{ \"id\": 3.0, \"data\": \"c\" }", + "{ \"id\": 4.0, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode( + () -> decimalPrecision4DF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged decimal precision", + ImmutableList.of( + row(new BigDecimal("1.00"), "a"), + row(new BigDecimal("2.00"), "b"), + row(new BigDecimal("3.00"), "c"), + row(new BigDecimal("4.00"), "d")), + sql("select * from %s order by id", tableName)); + + // verify the decimal column precision did not change + Type idFieldType = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id").type(); + assertThat(idFieldType.typeId()).isEqualTo(Type.TypeID.DECIMAL); + Types.DecimalType decimalType = (Types.DecimalType) idFieldType; + assertThat(decimalType.precision()).isEqualTo(6); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java index 1c87886241bf..7404b18d14b2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java @@ -19,13 +19,16 @@ package org.apache.iceberg.spark.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.math.BigDecimal; import java.util.List; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; @@ -244,4 +247,135 @@ public void testMergeSchemaSparkConfiguration() throws Exception { row(4L, "d", 140000.56F)), sql("select * from %s order by id", tableName)); } + + @TestTemplate + public void testMergeSchemaIgnoreCastingLongToInt() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset bigintDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + bigintDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with long column", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + + Dataset intDF = + jsonToDF( + "id int, data string", + "{ \"id\": 3, \"data\": \"c\" }", + "{ \"id\": 4, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode(() -> intDF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged long column type", + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d")), + sql("select * from %s order by id", tableName)); + + // verify the column type did not change + Types.NestedField idField = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id"); + assertThat(idField.type().typeId()).isEqualTo(Type.TypeID.LONG); + } + + @TestTemplate + public void testMergeSchemaIgnoreCastingDoubleToFloat() throws Exception { + removeTables(); + sql("CREATE TABLE %s (id double, data string) USING iceberg", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset doubleDF = + jsonToDF( + "id double, data string", + "{ \"id\": 1.0, \"data\": \"a\" }", + "{ \"id\": 2.0, \"data\": \"b\" }"); + + doubleDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with double column", + ImmutableList.of(row(1.0, "a"), row(2.0, "b")), + sql("select * from %s order by id", tableName)); + + Dataset floatDF = + jsonToDF( + "id float, data string", + "{ \"id\": 3.0, \"data\": \"c\" }", + "{ \"id\": 4.0, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode(() -> floatDF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged double column type", + ImmutableList.of(row(1.0, "a"), row(2.0, "b"), row(3.0, "c"), row(4.0, "d")), + sql("select * from %s order by id", tableName)); + + // verify the column type did not change + Types.NestedField idField = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id"); + assertThat(idField.type().typeId()).isEqualTo(Type.TypeID.DOUBLE); + } + + @TestTemplate + public void testMergeSchemaIgnoreCastingDecimalToDecimalWithNarrowerPrecision() throws Exception { + removeTables(); + sql("CREATE TABLE %s (id decimal(6,2), data string) USING iceberg", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset decimalPrecision6DF = + jsonToDF( + "id decimal(6,2), data string", + "{ \"id\": 1.0, \"data\": \"a\" }", + "{ \"id\": 2.0, \"data\": \"b\" }"); + + decimalPrecision6DF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with decimal column with precision 6", + ImmutableList.of(row(new BigDecimal("1.00"), "a"), row(new BigDecimal("2.00"), "b")), + sql("select * from %s order by id", tableName)); + + Dataset decimalPrecision4DF = + jsonToDF( + "id decimal(4,2), data string", + "{ \"id\": 3.0, \"data\": \"c\" }", + "{ \"id\": 4.0, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode( + () -> decimalPrecision4DF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged decimal precision", + ImmutableList.of( + row(new BigDecimal("1.00"), "a"), + row(new BigDecimal("2.00"), "b"), + row(new BigDecimal("3.00"), "c"), + row(new BigDecimal("4.00"), "d")), + sql("select * from %s order by id", tableName)); + + // verify the decimal column precision did not change + Type idFieldType = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id").type(); + assertThat(idFieldType.typeId()).isEqualTo(Type.TypeID.DECIMAL); + Types.DecimalType decimalType = (Types.DecimalType) idFieldType; + assertThat(decimalType.precision()).isEqualTo(6); + } } From 307593ffd99752b2d62cc91f4928285fc0c62b75 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 15 Nov 2024 18:19:52 +0800 Subject: [PATCH 143/313] Docs: Fix level of Deletion Vectors (#11547) --- format/spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index 631008c5daf1..9d8157ae302d 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1016,7 +1016,7 @@ Row-level delete files and deletion vectors are tracked by manifests. A separate Both position and equality delete files allow encoding deleted row values with a delete. This can be used to reconstruct a stream of changes to a table. -### Deletion Vectors +#### Deletion Vectors Deletion vectors identify deleted rows of a file by encoding deleted positions in a bitmap. A set bit at position P indicates that the row at position P is deleted. From 50d310aef17908f03f595d520cd751527483752a Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Fri, 15 Nov 2024 08:50:29 -0700 Subject: [PATCH 144/313] API, Core: Replace deprecated ContentFile#path usage with location (#11550) --- .../java/org/apache/iceberg/DeleteFiles.java | 2 +- .../iceberg/encryption/EncryptingFileIO.java | 7 +- .../java/org/apache/iceberg/io/FileIO.java | 8 +- .../iceberg/BaseChangelogContentScanTask.java | 4 +- .../apache/iceberg/BaseContentScanTask.java | 2 +- .../apache/iceberg/BaseOverwriteFiles.java | 2 +- .../java/org/apache/iceberg/CatalogUtil.java | 2 +- .../org/apache/iceberg/ContentFileParser.java | 2 +- .../java/org/apache/iceberg/DataFiles.java | 2 +- .../java/org/apache/iceberg/FileMetadata.java | 2 +- .../iceberg/IncrementalFileCleanup.java | 4 +- .../org/apache/iceberg/ManifestFiles.java | 2 +- .../apache/iceberg/ManifestFilterManager.java | 6 +- .../iceberg/MergingSnapshotProducer.java | 16 ++-- .../iceberg/SplitPositionDeletesScanTask.java | 2 +- .../java/org/apache/iceberg/V1Metadata.java | 2 +- .../java/org/apache/iceberg/V2Metadata.java | 2 +- .../java/org/apache/iceberg/V3Metadata.java | 2 +- .../actions/BaseRewriteDataFilesAction.java | 3 +- .../RewriteDataFilesCommitManager.java | 4 +- .../RewritePositionDeletesCommitManager.java | 3 +- .../encryption/InputFilesDecryptor.java | 2 +- .../java/org/apache/iceberg/hadoop/Util.java | 4 +- .../org/apache/iceberg/io/BaseTaskWriter.java | 2 +- .../apache/iceberg/util/PartitionUtil.java | 2 +- .../apache/iceberg/DataTableScanTestBase.java | 27 +++--- .../iceberg/DeleteFileIndexTestBase.java | 93 ++++++++++--------- .../apache/iceberg/FileGenerationUtil.java | 2 +- .../iceberg/MetadataTableScanTestBase.java | 2 +- .../ScanPlanningAndReportingTestBase.java | 2 +- .../java/org/apache/iceberg/TestBase.java | 30 +++--- .../TestBaseIncrementalChangelogScan.java | 22 ++--- .../org/apache/iceberg/TestBatchScans.java | 10 +- .../apache/iceberg/TestContentFileParser.java | 2 +- .../iceberg/TestEntriesMetadataTable.java | 6 +- .../org/apache/iceberg/TestFindFiles.java | 5 +- .../iceberg/TestIncrementalDataTableScan.java | 2 +- .../iceberg/TestManifestEncryption.java | 2 +- .../apache/iceberg/TestManifestReader.java | 2 +- .../iceberg/TestManifestReaderStats.java | 12 ++- .../iceberg/TestManifestWriterVersions.java | 2 +- .../iceberg/TestMetadataTableScans.java | 14 +-- ...adataTableScansWithPartitionEvolution.java | 6 +- .../apache/iceberg/TestMicroBatchBuilder.java | 2 +- .../apache/iceberg/TestRemoveSnapshots.java | 32 +++---- .../apache/iceberg/TestRewriteManifests.java | 18 ++-- .../java/org/apache/iceberg/TestRowDelta.java | 22 ++--- .../iceberg/TestSequenceNumberForV2Table.java | 4 +- .../iceberg/TestV1ToV2RowDeltaDelete.java | 17 ++-- .../apache/iceberg/catalog/CatalogTests.java | 2 +- .../hadoop/TestCatalogUtilDropTable.java | 3 +- .../hadoop/TestTableSerialization.java | 4 +- 52 files changed, 218 insertions(+), 216 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/DeleteFiles.java b/api/src/main/java/org/apache/iceberg/DeleteFiles.java index 8a396920e03b..682ebdd49f0c 100644 --- a/api/src/main/java/org/apache/iceberg/DeleteFiles.java +++ b/api/src/main/java/org/apache/iceberg/DeleteFiles.java @@ -51,7 +51,7 @@ public interface DeleteFiles extends SnapshotUpdate { * @return this for method chaining */ default DeleteFiles deleteFile(DataFile file) { - deleteFile(file.path()); + deleteFile(file.location()); return this; } diff --git a/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java index 0203361844a5..d3de7b1f84a3 100644 --- a/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java @@ -93,10 +93,9 @@ public InputFile newInputFile(DeleteFile file) { private InputFile newInputFile(ContentFile file) { if (file.keyMetadata() != null) { - return newDecryptingInputFile( - file.path().toString(), file.fileSizeInBytes(), file.keyMetadata()); + return newDecryptingInputFile(file.location(), file.fileSizeInBytes(), file.keyMetadata()); } else { - return newInputFile(file.path().toString(), file.fileSizeInBytes()); + return newInputFile(file.location(), file.fileSizeInBytes()); } } @@ -148,7 +147,7 @@ public void close() { } private SimpleEncryptedInputFile wrap(ContentFile file) { - InputFile encryptedInputFile = io.newInputFile(file.path().toString(), file.fileSizeInBytes()); + InputFile encryptedInputFile = io.newInputFile(file.location(), file.fileSizeInBytes()); return new SimpleEncryptedInputFile(encryptedInputFile, toKeyMetadata(file.keyMetadata())); } diff --git a/api/src/main/java/org/apache/iceberg/io/FileIO.java b/api/src/main/java/org/apache/iceberg/io/FileIO.java index de4bc2e12a81..f5404b9e5a78 100644 --- a/api/src/main/java/org/apache/iceberg/io/FileIO.java +++ b/api/src/main/java/org/apache/iceberg/io/FileIO.java @@ -50,16 +50,16 @@ default InputFile newInputFile(DataFile file) { Preconditions.checkArgument( file.keyMetadata() == null, "Cannot decrypt data file: %s (use EncryptingFileIO)", - file.path()); - return newInputFile(file.path().toString(), file.fileSizeInBytes()); + file.location()); + return newInputFile(file.location(), file.fileSizeInBytes()); } default InputFile newInputFile(DeleteFile file) { Preconditions.checkArgument( file.keyMetadata() == null, "Cannot decrypt delete file: %s (use EncryptingFileIO)", - file.path()); - return newInputFile(file.path().toString(), file.fileSizeInBytes()); + file.location()); + return newInputFile(file.location(), file.fileSizeInBytes()); } default InputFile newInputFile(ManifestFile manifest) { diff --git a/core/src/main/java/org/apache/iceberg/BaseChangelogContentScanTask.java b/core/src/main/java/org/apache/iceberg/BaseChangelogContentScanTask.java index 64c8cae08019..924148214ba0 100644 --- a/core/src/main/java/org/apache/iceberg/BaseChangelogContentScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseChangelogContentScanTask.java @@ -56,7 +56,7 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("change_ordinal", changeOrdinal) .add("commit_snapshot_id", commitSnapshotId) - .add("file", file().path()) + .add("file", file().location()) .add("partition_data", file().partition()) .add("residual", residual()) .toString(); @@ -142,7 +142,7 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("change_ordinal", changeOrdinal()) .add("commit_snapshot_id", commitSnapshotId()) - .add("file", file().path()) + .add("file", file().location()) .add("partition_data", file().partition()) .add("offset", offset) .add("length", length) diff --git a/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java b/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java index 960c04cc0f37..53c2875a7043 100644 --- a/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseContentScanTask.java @@ -117,7 +117,7 @@ public Iterable split(long targetSplitSize) { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("file", file().path()) + .add("file", file().location()) .add("partition_data", file().partition()) .add("residual", residual()) .toString(); diff --git a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java index 16fbc0dd1ebc..3bf44244862e 100644 --- a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java @@ -141,7 +141,7 @@ protected void validate(TableMetadata base, Snapshot parent) { && (strict.eval(file.partition()) || metrics.eval(file)), "Cannot append file with rows that do not match filter: %s: %s", rowFilter, - file.path()); + file.location()); } } diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 609e94b7b150..37d9ad86e16d 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -175,7 +175,7 @@ private static void deleteFiles(FileIO io, Set allManifests) { for (ManifestEntry entry : reader.entries()) { // intern the file path because the weak key map uses identity (==) instead of // equals - String path = entry.file().path().toString().intern(); + String path = entry.file().location().intern(); Boolean alreadyDeleted = deletedFiles.putIfAbsent(path, true); if (alreadyDeleted == null || !alreadyDeleted) { pathsToDelete.add(path); diff --git a/core/src/main/java/org/apache/iceberg/ContentFileParser.java b/core/src/main/java/org/apache/iceberg/ContentFileParser.java index e6d7c8043f3f..1be06cb42602 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileParser.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileParser.java @@ -83,7 +83,7 @@ static void toJson(ContentFile contentFile, PartitionSpec spec, JsonGenerator generator.writeNumberField(SPEC_ID, contentFile.specId()); generator.writeStringField(CONTENT, contentFile.content().name()); - generator.writeStringField(FILE_PATH, contentFile.path().toString()); + generator.writeStringField(FILE_PATH, contentFile.location()); generator.writeStringField(FILE_FORMAT, contentFile.format().name()); if (contentFile.partition() != null) { diff --git a/core/src/main/java/org/apache/iceberg/DataFiles.java b/core/src/main/java/org/apache/iceberg/DataFiles.java index 5c7c1a646cc9..0404f2da52b4 100644 --- a/core/src/main/java/org/apache/iceberg/DataFiles.java +++ b/core/src/main/java/org/apache/iceberg/DataFiles.java @@ -186,7 +186,7 @@ public Builder copy(DataFile toCopy) { specId == toCopy.specId(), "Cannot copy a DataFile with a different spec"); this.partitionData = copyPartitionData(spec, toCopy.partition(), partitionData); } - this.filePath = toCopy.path().toString(); + this.filePath = toCopy.location(); this.format = toCopy.format(); this.recordCount = toCopy.recordCount(); this.fileSizeInBytes = toCopy.fileSizeInBytes(); diff --git a/core/src/main/java/org/apache/iceberg/FileMetadata.java b/core/src/main/java/org/apache/iceberg/FileMetadata.java index 7bb8d886dd16..15936ec8760c 100644 --- a/core/src/main/java/org/apache/iceberg/FileMetadata.java +++ b/core/src/main/java/org/apache/iceberg/FileMetadata.java @@ -94,7 +94,7 @@ public Builder copy(DeleteFile toCopy) { this.partitionData = DataFiles.copyPartitionData(spec, toCopy.partition(), partitionData); } this.content = toCopy.content(); - this.filePath = toCopy.path().toString(); + this.filePath = toCopy.location(); this.format = toCopy.format(); this.recordCount = toCopy.recordCount(); this.fileSizeInBytes = toCopy.fileSizeInBytes(); diff --git a/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java b/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java index 60ad46e8e864..e1648514ef0e 100644 --- a/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java +++ b/core/src/main/java/org/apache/iceberg/IncrementalFileCleanup.java @@ -293,7 +293,7 @@ private Set findFilesToDelete( if (entry.status() == ManifestEntry.Status.DELETED && !validIds.contains(entry.snapshotId())) { // use toString to ensure the path will not change (Utf8 is reused) - filesToDelete.add(entry.file().path().toString()); + filesToDelete.add(entry.file().location()); } } } catch (IOException e) { @@ -317,7 +317,7 @@ private Set findFilesToDelete( // delete any ADDED file from manifests that were reverted if (entry.status() == ManifestEntry.Status.ADDED) { // use toString to ensure the path will not change (Utf8 is reused) - filesToDelete.add(entry.file().path().toString()); + filesToDelete.add(entry.file().location()); } } } catch (IOException e) { diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index f630bb3eb743..c9f6b783b93f 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -96,7 +96,7 @@ public static void dropCache(FileIO fileIO) { public static CloseableIterable readPaths(ManifestFile manifest, FileIO io) { return CloseableIterable.transform( read(manifest, io, null).select(ImmutableList.of("file_path")).liveEntries(), - entry -> entry.file().path().toString()); + entry -> entry.file().location()); } /** diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index adc5e1b7d3c2..564e27a937a1 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -246,7 +246,7 @@ private void validateRequiredDeletes(ManifestFile... manifests) { CharSequenceSet deletedFilePaths = deletedFiles.stream() - .map(ContentFile::path) + .map(ContentFile::location) .collect(Collectors.toCollection(CharSequenceSet::empty)); ValidationException.check( @@ -388,7 +388,7 @@ private boolean manifestHasDeletedFiles( for (ManifestEntry entry : reader.liveEntries()) { F file = entry.file(); boolean markedForDelete = - deletePaths.contains(file.path()) + deletePaths.contains(file.location()) || deleteFiles.contains(file) || dropPartitions.contains(file.specId(), file.partition()) || (isDelete @@ -436,7 +436,7 @@ private ManifestFile filterManifestWithDeletedFiles( entry -> { F file = entry.file(); boolean markedForDelete = - deletePaths.contains(file.path()) + deletePaths.contains(file.location()) || deleteFiles.contains(file) || dropPartitions.contains(file.specId(), file.partition()) || (isDelete diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 6198ad00f680..41f0ad00178c 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -355,7 +355,7 @@ protected void validateAddedDataFiles( "Found conflicting files that can contain records matching partitions %s: %s", partitionSet, Iterators.toString( - Iterators.transform(conflicts, entry -> entry.file().path().toString()))); + Iterators.transform(conflicts, entry -> entry.file().location().toString()))); } } catch (IOException e) { @@ -386,7 +386,7 @@ protected void validateAddedDataFiles( "Found conflicting files that can contain records matching %s: %s", conflictDetectionFilter, Iterators.toString( - Iterators.transform(conflicts, entry -> entry.file().path().toString()))); + Iterators.transform(conflicts, entry -> entry.file().location().toString()))); } } catch (IOException e) { @@ -550,7 +550,7 @@ protected void validateNoNewDeleteFiles( deletes.isEmpty(), "Found new conflicting delete files that can apply to records matching %s: %s", dataFilter, - Iterables.transform(deletes.referencedDeleteFiles(), ContentFile::path)); + Iterables.transform(deletes.referencedDeleteFiles(), ContentFile::location)); } /** @@ -570,7 +570,7 @@ protected void validateNoNewDeleteFiles( deletes.isEmpty(), "Found new conflicting delete files that can apply to records matching %s: %s", partitionSet, - Iterables.transform(deletes.referencedDeleteFiles(), ContentFile::path)); + Iterables.transform(deletes.referencedDeleteFiles(), ContentFile::location)); } /** @@ -628,7 +628,7 @@ protected void validateDeletedDataFiles( "Found conflicting deleted files that can contain records matching %s: %s", dataFilter, Iterators.toString( - Iterators.transform(conflicts, entry -> entry.file().path().toString()))); + Iterators.transform(conflicts, entry -> entry.file().location().toString()))); } } catch (IOException e) { @@ -657,7 +657,7 @@ protected void validateDeletedDataFiles( "Found conflicting deleted files that can apply to records matching %s: %s", partitionSet, Iterators.toString( - Iterators.transform(conflicts, entry -> entry.file().path().toString()))); + Iterators.transform(conflicts, entry -> entry.file().location().toString()))); } } catch (IOException e) { @@ -783,7 +783,7 @@ protected void validateDataFilesExist( entry -> entry.status() != ManifestEntry.Status.ADDED && newSnapshots.contains(entry.snapshotId()) - && requiredDataFiles.contains(entry.file().path())) + && requiredDataFiles.contains(entry.file().location())) .specsById(base.specsById()) .ignoreExisting(); @@ -797,7 +797,7 @@ protected void validateDataFilesExist( throw new ValidationException( "Cannot commit, missing data files: %s", Iterators.toString( - Iterators.transform(deletes, entry -> entry.file().path().toString()))); + Iterators.transform(deletes, entry -> entry.file().location().toString()))); } } catch (IOException e) { diff --git a/core/src/main/java/org/apache/iceberg/SplitPositionDeletesScanTask.java b/core/src/main/java/org/apache/iceberg/SplitPositionDeletesScanTask.java index 68c7d5f9fd88..1ec3599a9b84 100644 --- a/core/src/main/java/org/apache/iceberg/SplitPositionDeletesScanTask.java +++ b/core/src/main/java/org/apache/iceberg/SplitPositionDeletesScanTask.java @@ -80,7 +80,7 @@ public SplitPositionDeletesScanTask merge(org.apache.iceberg.ScanTask other) { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("file", file().path()) + .add("file", file().location()) .add("partition_data", file().partition()) .add("offset", offset) .add("length", length) diff --git a/core/src/main/java/org/apache/iceberg/V1Metadata.java b/core/src/main/java/org/apache/iceberg/V1Metadata.java index 78fa81cebe52..c5319a0bc100 100644 --- a/core/src/main/java/org/apache/iceberg/V1Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V1Metadata.java @@ -351,7 +351,7 @@ IndexedDataFile wrap(DataFile file) { public Object get(int pos) { switch (pos) { case 0: - return wrapped.path().toString(); + return wrapped.location(); case 1: return wrapped.format() != null ? wrapped.format().toString() : null; case 2: diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index 20b2169b8dad..bb715385610b 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -420,7 +420,7 @@ public Object get(int pos) { case 0: return wrapped.content().id(); case 1: - return wrapped.path().toString(); + return wrapped.location(); case 2: return wrapped.format() != null ? wrapped.format().toString() : null; case 3: diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java index 70461ac74a70..12f4a2058748 100644 --- a/core/src/main/java/org/apache/iceberg/V3Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -422,7 +422,7 @@ public Object get(int pos) { case 0: return wrapped.content().id(); case 1: - return wrapped.path().toString(); + return wrapped.location(); case 2: return wrapped.format() != null ? wrapped.format().toString() : null; case 3: diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesAction.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesAction.java index c0f2fc6174b3..f437b3afef1c 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesAction.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesAction.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.stream.Collectors; import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; @@ -311,7 +312,7 @@ private void replaceDataFiles( } catch (Exception e) { if (e instanceof CleanableFailure) { LOG.warn("Failed to commit rewrite, cleaning up rewritten files", e); - Tasks.foreach(Iterables.transform(addedDataFiles, f -> f.path().toString())) + Tasks.foreach(Iterables.transform(addedDataFiles, ContentFile::location)) .noRetry() .suppressFailureWhenFinished() .onFailure((location, exc) -> LOG.warn("Failed to delete: {}", location, exc)) diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java index 03d23231c0f1..777e2ce630c6 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java @@ -105,8 +105,8 @@ public void abortFileGroup(RewriteFileGroup fileGroup) { Tasks.foreach(fileGroup.addedFiles()) .noRetry() .suppressFailureWhenFinished() - .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc)) - .run(dataFile -> table.io().deleteFile(dataFile.path().toString())); + .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.location(), exc)) + .run(dataFile -> table.io().deleteFile(dataFile.location())); } public void commitOrClean(Set rewriteGroups) { diff --git a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java index b1322d5e58b4..3a75c87396c3 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java @@ -21,6 +21,7 @@ import java.util.Map; import java.util.Set; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.RewriteFiles; import org.apache.iceberg.Table; @@ -89,7 +90,7 @@ public void abort(RewritePositionDeletesGroup fileGroup) { fileGroup.addedDeleteFiles() != null, "Cannot abort a fileGroup that was not rewritten"); Iterable filePaths = - Iterables.transform(fileGroup.addedDeleteFiles(), f -> f.path().toString()); + Iterables.transform(fileGroup.addedDeleteFiles(), ContentFile::location); CatalogUtil.deleteFiles(table.io(), filePaths, "position delete", true); } diff --git a/core/src/main/java/org/apache/iceberg/encryption/InputFilesDecryptor.java b/core/src/main/java/org/apache/iceberg/encryption/InputFilesDecryptor.java index 158e61971da2..44cd38c58621 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/InputFilesDecryptor.java +++ b/core/src/main/java/org/apache/iceberg/encryption/InputFilesDecryptor.java @@ -59,7 +59,7 @@ public InputFilesDecryptor( public InputFile getInputFile(FileScanTask task) { Preconditions.checkArgument(!task.isDataTask(), "Invalid task type"); - return decryptedInputFiles.get(task.file().path().toString()); + return decryptedInputFiles.get(task.file().location()); } public InputFile getInputFile(String location) { diff --git a/core/src/main/java/org/apache/iceberg/hadoop/Util.java b/core/src/main/java/org/apache/iceberg/hadoop/Util.java index ce774d6f259b..e7b6f299a2a1 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/Util.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/Util.java @@ -62,7 +62,7 @@ public static FileSystem getFs(Path path, Configuration conf) { public static String[] blockLocations(CombinedScanTask task, Configuration conf) { Set locationSets = Sets.newHashSet(); for (FileScanTask f : task.files()) { - Path path = new Path(f.file().path().toString()); + Path path = new Path(f.file().location()); try { FileSystem fs = path.getFileSystem(conf); for (BlockLocation b : fs.getFileBlockLocations(path, f.start(), f.length())) { @@ -104,7 +104,7 @@ public static boolean mayHaveBlockLocations(FileIO io, String location) { } private static String[] blockLocations(FileIO io, ContentScanTask task) { - String location = task.file().path().toString(); + String location = task.file().location(); if (usesHadoopFileIO(io, location)) { InputFile inputFile = io.newInputFile(location); if (inputFile instanceof HadoopInputFile) { diff --git a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index 968db0ab538b..0834c7156a9c 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -92,7 +92,7 @@ public void abort() throws IOException { .executeWith(ThreadPools.getWorkerPool()) .throwFailureWhenFinished() .noRetry() - .run(file -> io.deleteFile(file.path().toString())); + .run(file -> io.deleteFile(file.location())); } @Override diff --git a/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java b/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java index 83735939317b..411d401075d6 100644 --- a/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java @@ -56,7 +56,7 @@ private PartitionUtil() {} // add _file idToConstant.put( MetadataColumns.FILE_PATH.fieldId(), - convertConstant.apply(Types.StringType.get(), task.file().path())); + convertConstant.apply(Types.StringType.get(), task.file().location())); // add _spec_id idToConstant.put( diff --git a/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java b/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java index 8bd7a5ea70d9..e4cf74a15c09 100644 --- a/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java @@ -108,10 +108,10 @@ public void testScanFromBranchTip() throws IOException { ScanT testBranchScan = useRef(newScan(), "testBranch"); validateExpectedFileScanTasks( - testBranchScan, ImmutableList.of(FILE_A.path(), FILE_B.path(), FILE_C.path())); + testBranchScan, ImmutableList.of(FILE_A.location(), FILE_B.location(), FILE_C.location())); ScanT mainScan = newScan(); - validateExpectedFileScanTasks(mainScan, ImmutableList.of(FILE_A.path(), FILE_D.path())); + validateExpectedFileScanTasks(mainScan, ImmutableList.of(FILE_A.location(), FILE_D.location())); } @TestTemplate @@ -120,10 +120,10 @@ public void testScanFromTag() throws IOException { table.manageSnapshots().createTag("tagB", table.currentSnapshot().snapshotId()).commit(); table.newFastAppend().appendFile(FILE_C).commit(); ScanT tagScan = useRef(newScan(), "tagB"); - validateExpectedFileScanTasks(tagScan, ImmutableList.of(FILE_A.path(), FILE_B.path())); + validateExpectedFileScanTasks(tagScan, ImmutableList.of(FILE_A.location(), FILE_B.location())); ScanT mainScan = newScan(); validateExpectedFileScanTasks( - mainScan, ImmutableList.of(FILE_A.path(), FILE_B.path(), FILE_C.path())); + mainScan, ImmutableList.of(FILE_A.location(), FILE_B.location(), FILE_C.location())); } @TestTemplate @@ -196,9 +196,10 @@ private void validateExpectedFileScanTasks( List actualFiles = Lists.newArrayList(); for (T task : scanTasks) { DataFile dataFile = ((FileScanTask) task).file(); - actualFiles.add(dataFile.path()); + actualFiles.add(dataFile.location()); if (fileToManifest != null) { - assertThat(fileToManifest.get(dataFile.path())).isEqualTo(dataFile.manifestLocation()); + assertThat(fileToManifest.get(dataFile.location())) + .isEqualTo(dataFile.manifestLocation()); } } @@ -231,12 +232,12 @@ public void testSequenceNumbersThroughPlanFiles() { DataFile file = fileScanTask.file(); long expectedDataSequenceNumber = 0L; long expectedDeleteSequenceNumber = 0L; - if (file.path().equals(dataFile1.path())) { + if (file.location().equals(dataFile1.location())) { expectedDataSequenceNumber = 1L; expectedDeleteSequenceNumber = 3L; } - if (file.path().equals(dataFile2.path())) { + if (file.location().equals(dataFile2.location())) { expectedDataSequenceNumber = 2L; expectedDeleteSequenceNumber = 4L; } @@ -274,9 +275,9 @@ public void testManifestLocationsInScan() throws IOException { .collect(Collectors.toList()) .get(0); CharSequenceMap fileToManifest = CharSequenceMap.create(); - fileToManifest.put(FILE_A.path(), firstDataManifest.path()); - fileToManifest.put(FILE_B.path(), secondDataManifest.path()); - fileToManifest.put(FILE_C.path(), secondDataManifest.path()); + fileToManifest.put(FILE_A.location(), firstDataManifest.path()); + fileToManifest.put(FILE_B.location(), secondDataManifest.path()); + fileToManifest.put(FILE_C.location(), secondDataManifest.path()); validateExpectedFileScanTasks(newScan(), fileToManifest.keySet(), fileToManifest); } @@ -290,9 +291,9 @@ public void testManifestLocationsInScanWithDeleteFiles() throws IOException { DeleteFile deleteFile = newDeleteFile("data_bucket=0"); table.newRowDelta().addDeletes(deleteFile).commit(); CharSequenceMap fileToManifest = CharSequenceMap.create(); - fileToManifest.put(FILE_A.path(), firstManifest.path()); + fileToManifest.put(FILE_A.location(), firstManifest.path()); ScanT scan = newScan(); - validateExpectedFileScanTasks(scan, ImmutableList.of(FILE_A.path()), fileToManifest); + validateExpectedFileScanTasks(scan, ImmutableList.of(FILE_A.location()), fileToManifest); List deletes = Lists.newArrayList(); try (CloseableIterable scanTasks = scan.planFiles()) { for (T task : scanTasks) { diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java index 481422457b73..ab3cb563c175 100644 --- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java @@ -275,13 +275,13 @@ public void testUnpartitionedTableScan() throws IOException { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(unpartitionedFile.path()); + .isEqualTo(unpartitionedFile.location()); assertThat(task.deletes()).as("Should have one associated delete file").hasSize(1); - assertThat(task.deletes().get(0).path()) + assertThat(task.deletes().get(0).location()) .as("Should have expected delete file") - .isEqualTo(unpartitionedPosDeletes.path()); + .isEqualTo(unpartitionedPosDeletes.location()); // add a second delete file DeleteFile unpartitionedEqDeletes = unpartitionedEqDeletes(unpartitioned.spec()); @@ -289,13 +289,14 @@ public void testUnpartitionedTableScan() throws IOException { tasks = Lists.newArrayList(newScan(unpartitioned).planFiles().iterator()); task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(unpartitionedFile.path()); + .isEqualTo(unpartitionedFile.location()); assertThat(task.deletes()).as("Should have two associated delete files").hasSize(2); - assertThat(Sets.newHashSet(Iterables.transform(task.deletes(), ContentFile::path))) + assertThat(Sets.newHashSet(Iterables.transform(task.deletes(), ContentFile::location))) .as("Should have expected delete files") - .isEqualTo(Sets.newHashSet(unpartitionedPosDeletes.path(), unpartitionedEqDeletes.path())); + .isEqualTo( + Sets.newHashSet(unpartitionedPosDeletes.location(), unpartitionedEqDeletes.location())); } @TestTemplate @@ -308,13 +309,13 @@ public void testPartitionedTableWithPartitionPosDeletes() { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(FILE_A.path()); + .isEqualTo(FILE_A.location()); assertThat(task.deletes()).as("Should have one associated delete file").hasSize(1); - assertThat(task.deletes().get(0).path()) + assertThat(task.deletes().get(0).location()) .as("Should have only pos delete file") - .isEqualTo(fileADeletes().path()); + .isEqualTo(fileADeletes().location()); } @TestTemplate @@ -327,13 +328,13 @@ public void testPartitionedTableWithPartitionEqDeletes() { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(FILE_A.path()); + .isEqualTo(FILE_A.location()); assertThat(task.deletes()).as("Should have one associated delete file").hasSize(1); - assertThat(task.deletes().get(0).path()) + assertThat(task.deletes().get(0).location()) .as("Should have only pos delete file") - .isEqualTo(FILE_A_EQ_1.path()); + .isEqualTo(FILE_A_EQ_1.location()); } @TestTemplate @@ -346,9 +347,9 @@ public void testPartitionedTableWithUnrelatedPartitionDeletes() { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(FILE_B.path()); + .isEqualTo(FILE_B.location()); assertThat(task.deletes()).as("Should have no delete files to apply").hasSize(0); } @@ -364,9 +365,9 @@ public void testPartitionedTableWithOlderPartitionDeletes() { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(FILE_A.path()); + .isEqualTo(FILE_A.location()); assertThat(task.deletes()).as("Should have no delete files to apply").hasSize(0); } @@ -391,13 +392,13 @@ public void testPartitionedTableScanWithGlobalDeletes() { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(FILE_A.path()); + .isEqualTo(FILE_A.location()); assertThat(task.deletes()).as("Should have one associated delete file").hasSize(1); - assertThat(task.deletes().get(0).path()) + assertThat(task.deletes().get(0).location()) .as("Should have expected delete file") - .isEqualTo(unpartitionedEqDeletes.path()); + .isEqualTo(unpartitionedEqDeletes.location()); } @TestTemplate @@ -423,13 +424,13 @@ public void testPartitionedTableScanWithGlobalAndPartitionDeletes() { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(FILE_A.path()); + .isEqualTo(FILE_A.location()); assertThat(task.deletes()).as("Should have two associated delete files").hasSize(2); - assertThat(Sets.newHashSet(Iterables.transform(task.deletes(), ContentFile::path))) + assertThat(Sets.newHashSet(Iterables.transform(task.deletes(), ContentFile::location))) .as("Should have expected delete files") - .isEqualTo(Sets.newHashSet(unpartitionedEqDeletes.path(), FILE_A_EQ_1.path())); + .isEqualTo(Sets.newHashSet(unpartitionedEqDeletes.location(), FILE_A_EQ_1.location())); } @TestTemplate @@ -440,13 +441,13 @@ public void testPartitionedTableSequenceNumbers() { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(FILE_A.path()); + .isEqualTo(FILE_A.location()); assertThat(task.deletes()).as("Should have one associated delete file").hasSize(1); - assertThat(task.deletes().get(0).path()) + assertThat(task.deletes().get(0).location()) .as("Should have only pos delete file") - .isEqualTo(fileADeletes().path()); + .isEqualTo(fileADeletes().location()); } @TestTemplate @@ -481,13 +482,13 @@ public void testUnpartitionedTableSequenceNumbers() throws IOException { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(unpartitionedFile.path()); + .isEqualTo(unpartitionedFile.location()); assertThat(task.deletes()).as("Should have one associated delete file").hasSize(1); - assertThat(task.deletes().get(0).path()) + assertThat(task.deletes().get(0).location()) .as("Should have only pos delete file") - .isEqualTo(unpartitionedPosDeleteFile.path()); + .isEqualTo(unpartitionedPosDeleteFile.location()); } @TestTemplate @@ -548,13 +549,13 @@ public void testPartitionedTableWithExistingDeleteFile() { assertThat(tasks).as("Should have one task").hasSize(1); FileScanTask task = (FileScanTask) tasks.get(0); - assertThat(task.file().path()) + assertThat(task.file().location()) .as("Should have the correct data file path") - .isEqualTo(FILE_A.path()); + .isEqualTo(FILE_A.location()); assertThat(task.deletes()).as("Should have two associated delete files").hasSize(2); - assertThat(Sets.newHashSet(Iterables.transform(task.deletes(), ContentFile::path))) + assertThat(Sets.newHashSet(Iterables.transform(task.deletes(), ContentFile::location))) .as("Should have expected delete files") - .isEqualTo(Sets.newHashSet(FILE_A_EQ_1.path(), fileADeletes().path())); + .isEqualTo(Sets.newHashSet(FILE_A_EQ_1.location(), fileADeletes().location())); } @TestTemplate @@ -575,8 +576,10 @@ public void testPositionDeletesGroup() { // all files must be reported as referenced CharSequenceSet paths = - CharSequenceSet.of(Iterables.transform(group.referencedDeleteFiles(), ContentFile::path)); - assertThat(paths).contains(file1.path(), file2.path(), file3.path(), file4.path()); + CharSequenceSet.of( + Iterables.transform(group.referencedDeleteFiles(), ContentFile::location)); + assertThat(paths) + .contains(file1.location(), file2.location(), file3.location(), file4.location()); // position deletes are indexed by their data sequence numbers // so that position deletes can apply to data files added in the same snapshot @@ -609,8 +612,10 @@ public void testEqualityDeletesGroup() { // all files must be reported as referenced CharSequenceSet paths = - CharSequenceSet.of(Iterables.transform(group.referencedDeleteFiles(), ContentFile::path)); - assertThat(paths).contains(file1.path(), file2.path(), file3.path(), file4.path()); + CharSequenceSet.of( + Iterables.transform(group.referencedDeleteFiles(), ContentFile::location)); + assertThat(paths) + .contains(file1.location(), file2.location(), file3.location(), file4.location()); // equality deletes are indexed by data sequence number - 1 to apply to next snapshots assertThat(group.filter(0, FILE_A)).isEqualTo(new DeleteFile[] {file1, file2, file3, file4}); diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java index 4f85151c80da..5c6c2fe63f30 100644 --- a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -213,7 +213,7 @@ private static Metrics generatePositionDeleteMetrics(DataFile dataFile) { int fieldId = column.fieldId(); columnSizes.put(fieldId, generateColumnSize()); if (fieldId == MetadataColumns.DELETE_FILE_PATH.fieldId()) { - ByteBuffer bound = Conversions.toByteBuffer(Types.StringType.get(), dataFile.path()); + ByteBuffer bound = Conversions.toByteBuffer(Types.StringType.get(), dataFile.location()); lowerBounds.put(fieldId, bound); upperBounds.put(fieldId, bound); } diff --git a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java index 98d2d8f38af6..7b06b6560958 100644 --- a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java @@ -45,7 +45,7 @@ protected static List parameters() { protected Set scannedPaths(TableScan scan) { return StreamSupport.stream(scan.planFiles().spliterator(), false) - .map(t -> t.file().path().toString()) + .map(t -> t.file().location().toString()) .collect(Collectors.toSet()); } diff --git a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java index 80551f0a2247..f7c287f1dc04 100644 --- a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java @@ -241,7 +241,7 @@ public void scanningWithSkippedDataFiles() throws IOException { } assertThat(fileTasks) .singleElement() - .satisfies(task -> assertThat(task.file().path()).isEqualTo(FILE_D.path())); + .satisfies(task -> assertThat(task.file().location()).isEqualTo(FILE_D.location())); ScanReport scanReport = reporter.lastReport(); assertThat(scanReport).isNotNull(); diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 46a1518e877f..7437554f5a6f 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -490,9 +490,7 @@ void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber, DataFile snap.sequenceNumber(), entry.file().fileSequenceNumber().longValue()); } - assertThat(file.path().toString()) - .as("Path should match expected") - .isEqualTo(newPaths.next()); + assertThat(file.location()).as("Path should match expected").isEqualTo(newPaths.next()); assertThat(entry.snapshotId()).as("File's snapshot ID should match").isEqualTo(id); } @@ -508,11 +506,11 @@ void validateTableFiles(Table tbl, DataFile... expectedFiles) { void validateTableFiles(Table tbl, Collection expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DataFile file : expectedFiles) { - expectedFilePaths.add(file.path()); + expectedFilePaths.add(file.location()); } Set actualFilePaths = Sets.newHashSet(); for (FileScanTask task : tbl.newScan().planFiles()) { - actualFilePaths.add(task.file().path()); + actualFilePaths.add(task.file().location()); } assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); } @@ -520,11 +518,11 @@ void validateTableFiles(Table tbl, Collection expectedFiles) { void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DataFile file : expectedFiles) { - expectedFilePaths.add(file.path()); + expectedFilePaths.add(file.location()); } Set actualFilePaths = Sets.newHashSet(); for (FileScanTask task : tbl.newScan().useRef(ref).planFiles()) { - actualFilePaths.add(task.file().path()); + actualFilePaths.add(task.file().location()); } assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); } @@ -532,12 +530,12 @@ void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) { void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DeleteFile file : expectedFiles) { - expectedFilePaths.add(file.path()); + expectedFilePaths.add(file.location()); } Set actualFilePaths = Sets.newHashSet(); for (FileScanTask task : tbl.newScan().useRef(branch).planFiles()) { for (DeleteFile file : task.deletes()) { - actualFilePaths.add(file.path()); + actualFilePaths.add(file.location()); } } assertThat(actualFilePaths).as("Delete files should match").isEqualTo(expectedFilePaths); @@ -546,7 +544,7 @@ void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedF List paths(DataFile... dataFiles) { List paths = Lists.newArrayListWithExpectedSize(dataFiles.length); for (DataFile file : dataFiles) { - paths.add(file.path().toString()); + paths.add(file.location()); } return paths; } @@ -578,9 +576,7 @@ void validateManifest( validateManifestSequenceNumbers(entry, dataSeqs, fileSeqs); - assertThat(file.path().toString()) - .as("Path should match expected") - .isEqualTo(expected.path().toString()); + assertThat(file.location()).as("Path should match expected").isEqualTo(expected.location()); assertThat(entry.snapshotId()) .as("Snapshot ID should match expected ID") .isEqualTo(ids.next()); @@ -606,9 +602,7 @@ void validateDeleteManifest( validateManifestSequenceNumbers(entry, dataSeqs, fileSeqs); - assertThat(file.path().toString()) - .as("Path should match expected") - .isEqualTo(expected.path().toString()); + assertThat(file.location()).as("Path should match expected").isEqualTo(expected.location()); assertThat(entry.snapshotId()) .as("Snapshot ID should match expected ID") .isEqualTo(ids.next()); @@ -763,9 +757,7 @@ static void validateManifestEntries( DataFile file = entry.file(); DataFile expected = expectedFiles.next(); final ManifestEntry.Status expectedStatus = expectedStatuses.next(); - assertThat(file.path().toString()) - .as("Path should match expected") - .isEqualTo(expected.path().toString()); + assertThat(file.location()).as("Path should match expected").isEqualTo(expected.location()); assertThat(entry.snapshotId()) .as("Snapshot ID should match expected ID") .isEqualTo(ids.next()); diff --git a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java index 3b9488613367..3d6788a760b4 100644 --- a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java +++ b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java @@ -73,7 +73,7 @@ public void testDataFilters() { AddedRowsScanTask t1 = (AddedRowsScanTask) Iterables.getOnlyElement(tasks); assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(1); assertThat(t1.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap2.snapshotId()); - assertThat(t1.file().path()).as("Data file must match").isEqualTo(FILE_B.path()); + assertThat(t1.file().location()).as("Data file must match").isEqualTo(FILE_B.location()); assertThat(t1.deletes()).as("Must be no deletes").isEmpty(); }); } @@ -98,13 +98,13 @@ public void testOverwrites() { AddedRowsScanTask t1 = (AddedRowsScanTask) tasks.get(0); assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0); assertThat(t1.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap2.snapshotId()); - assertThat(t1.file().path()).as("Data file must match").isEqualTo(FILE_A2.path()); + assertThat(t1.file().location()).as("Data file must match").isEqualTo(FILE_A2.location()); assertThat(t1.deletes()).as("Must be no deletes").isEmpty(); DeletedDataFileScanTask t2 = (DeletedDataFileScanTask) tasks.get(1); assertThat(t2.changeOrdinal()).as("Ordinal must match").isEqualTo(0); assertThat(t2.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap2.snapshotId()); - assertThat(t2.file().path()).as("Data file must match").isEqualTo(FILE_A.path()); + assertThat(t2.file().location()).as("Data file must match").isEqualTo(FILE_A.location()); assertThat(t2.existingDeletes()).as("Must be no deletes").isEmpty(); } @@ -128,7 +128,7 @@ public void testFileDeletes() { DeletedDataFileScanTask t1 = (DeletedDataFileScanTask) Iterables.getOnlyElement(tasks); assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0); assertThat(t1.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap2.snapshotId()); - assertThat(t1.file().path()).as("Data file must match").isEqualTo(FILE_A.path()); + assertThat(t1.file().location()).as("Data file must match").isEqualTo(FILE_A.location()); assertThat(t1.existingDeletes()).as("Must be no deletes").isEmpty(); } @@ -161,7 +161,7 @@ public void testExistingEntriesInNewDataManifestsAreIgnored() { AddedRowsScanTask t1 = (AddedRowsScanTask) Iterables.getOnlyElement(tasks); assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0); assertThat(t1.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap3.snapshotId()); - assertThat(t1.file().path()).as("Data file must match").isEqualTo(FILE_C.path()); + assertThat(t1.file().location()).as("Data file must match").isEqualTo(FILE_C.location()); assertThat(t1.deletes()).as("Must be no deletes").isEmpty(); } @@ -202,19 +202,19 @@ public void testManifestRewritesAreIgnored() throws IOException { AddedRowsScanTask t1 = (AddedRowsScanTask) tasks.get(0); assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0); assertThat(t1.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap1.snapshotId()); - assertThat(t1.file().path()).as("Data file must match").isEqualTo(FILE_A.path()); + assertThat(t1.file().location()).as("Data file must match").isEqualTo(FILE_A.location()); assertThat(t1.deletes()).as("Must be no deletes").isEmpty(); AddedRowsScanTask t2 = (AddedRowsScanTask) tasks.get(1); assertThat(t2.changeOrdinal()).as("Ordinal must match").isEqualTo(1); assertThat(t2.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap2.snapshotId()); - assertThat(t2.file().path()).as("Data file must match").isEqualTo(FILE_B.path()); + assertThat(t2.file().location()).as("Data file must match").isEqualTo(FILE_B.location()); assertThat(t2.deletes()).as("Must be no deletes").isEmpty(); AddedRowsScanTask t3 = (AddedRowsScanTask) tasks.get(2); assertThat(t3.changeOrdinal()).as("Ordinal must match").isEqualTo(2); assertThat(t3.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap4.snapshotId()); - assertThat(t3.file().path()).as("Data file must match").isEqualTo(FILE_C.path()); + assertThat(t3.file().location()).as("Data file must match").isEqualTo(FILE_C.location()); assertThat(t3.deletes()).as("Must be no deletes").isEmpty(); } @@ -237,13 +237,13 @@ public void testDataFileRewrites() { AddedRowsScanTask t1 = (AddedRowsScanTask) tasks.get(0); assertThat(t1.changeOrdinal()).as("Ordinal must match").isEqualTo(0); assertThat(t1.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap1.snapshotId()); - assertThat(t1.file().path()).as("Data file must match").isEqualTo(FILE_A.path()); + assertThat(t1.file().location()).as("Data file must match").isEqualTo(FILE_A.location()); assertThat(t1.deletes()).as("Must be no deletes").isEmpty(); AddedRowsScanTask t2 = (AddedRowsScanTask) tasks.get(1); assertThat(t2.changeOrdinal()).as("Ordinal must match").isEqualTo(1); assertThat(t2.commitSnapshotId()).as("Snapshot must match").isEqualTo(snap2.snapshotId()); - assertThat(t2.file().path()).as("Data file must match").isEqualTo(FILE_B.path()); + assertThat(t2.file().location()).as("Data file must match").isEqualTo(FILE_B.location()); assertThat(t2.deletes()).as("Must be no deletes").isEmpty(); } @@ -282,6 +282,6 @@ private Comparator taskComparator() { } private String path(ChangelogScanTask task) { - return ((ContentScanTask) task).file().path().toString(); + return ((ContentScanTask) task).file().location().toString(); } } diff --git a/core/src/test/java/org/apache/iceberg/TestBatchScans.java b/core/src/test/java/org/apache/iceberg/TestBatchScans.java index 72cd00e0573d..bc707e51af43 100644 --- a/core/src/test/java/org/apache/iceberg/TestBatchScans.java +++ b/core/src/test/java/org/apache/iceberg/TestBatchScans.java @@ -51,12 +51,12 @@ public void testDataTableScan() { assertThat(tasks).hasSize(2); FileScanTask t1 = tasks.get(0).asFileScanTask(); - assertThat(FILE_A.path()).as("Task file must match").isEqualTo(t1.file().path()); + assertThat(FILE_A.location()).as("Task file must match").isEqualTo(t1.file().location()); V1Assert.assertEquals("Task deletes size must match", 0, t1.deletes().size()); V2Assert.assertEquals("Task deletes size must match", 1, t1.deletes().size()); FileScanTask t2 = tasks.get(1).asFileScanTask(); - assertThat(FILE_B.path()).as("Task file must match").isEqualTo(t2.file().path()); + assertThat(FILE_B.location()).as("Task file must match").isEqualTo(t2.file().location()); assertThat(t2.deletes()).as("Task deletes size must match").hasSize(0); List> taskGroups = planTaskGroups(scan); @@ -88,10 +88,10 @@ public void testFilesTableScan() { assertThat(tasks).as("Expected 2 tasks").hasSize(2); FileScanTask t1 = tasks.get(0).asFileScanTask(); - assertThat(manifestPaths).first().as("Task file must match").isEqualTo(t1.file().path()); + assertThat(manifestPaths).first().as("Task file must match").isEqualTo(t1.file().location()); FileScanTask t2 = tasks.get(1).asFileScanTask(); - assertThat(manifestPaths).element(1).as("Task file must match").isEqualTo(t2.file().path()); + assertThat(manifestPaths).element(1).as("Task file must match").isEqualTo(t2.file().location()); List> taskGroups = planTaskGroups(scan); assertThat(taskGroups).as("Expected 1 task group").hasSize(1); @@ -121,6 +121,6 @@ private List> planTaskGroups(BatchScan scan) { } private String path(ScanTask task) { - return ((ContentScanTask) task).file().path().toString(); + return ((ContentScanTask) task).file().location().toString(); } } diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index 0c98e8448745..759f2f8ecd0b 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -378,7 +378,7 @@ static void assertContentFileEquals( assertThat(actual.getClass()).isEqualTo(expected.getClass()); assertThat(actual.specId()).isEqualTo(expected.specId()); assertThat(actual.content()).isEqualTo(expected.content()); - assertThat(actual.path()).isEqualTo(expected.path()); + assertThat(actual.location()).isEqualTo(expected.location()); assertThat(actual.format()).isEqualTo(expected.format()); assertThat(actual.partition()) .usingComparator(Comparators.forType(spec.partitionType())) diff --git a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java index e061567e72a8..51e5c8e6570e 100644 --- a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java +++ b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java @@ -68,7 +68,7 @@ public void testEntriesTableScan() { .isEqualTo(expectedSchema.asStruct()); FileScanTask file = Iterables.getOnlyElement(scan.planFiles()); - assertThat(file.file().path()) + assertThat(file.file().location()) .as("Data file should be the table's manifest") .isEqualTo(table.currentSnapshot().allManifests(table.io()).get(0).path()); @@ -145,13 +145,13 @@ public void testEntriesTableWithDeleteManifests() { .isEqualTo(expectedSchema.asStruct()); List files = ImmutableList.copyOf(scan.planFiles()); - assertThat(files.get(0).file().path()) + assertThat(files.get(0).file().location()) .as("Data file should be the table's manifest") .isEqualTo(table.currentSnapshot().dataManifests(table.io()).get(0).path()); assertThat(files.get(0).file().recordCount()) .as("Should contain 2 data file records") .isEqualTo(2); - assertThat(files.get(1).file().path()) + assertThat(files.get(1).file().location()) .as("Delete file should be in the table manifest") .isEqualTo(table.currentSnapshot().deleteManifests(table.io()).get(0).path()); assertThat(files.get(1).file().recordCount()) diff --git a/core/src/test/java/org/apache/iceberg/TestFindFiles.java b/core/src/test/java/org/apache/iceberg/TestFindFiles.java index 191dbd384197..608358da3b49 100644 --- a/core/src/test/java/org/apache/iceberg/TestFindFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestFindFiles.java @@ -211,11 +211,10 @@ public void testNoSnapshot() { } private Set pathSet(DataFile... files) { - return Sets.newHashSet( - Iterables.transform(Arrays.asList(files), file -> file.path().toString())); + return Sets.newHashSet(Iterables.transform(Arrays.asList(files), ContentFile::location)); } private Set pathSet(Iterable files) { - return Sets.newHashSet(Iterables.transform(files, file -> file.path().toString())); + return Sets.newHashSet(Iterables.transform(files, ContentFile::location)); } } diff --git a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java index ecd6a14ffefb..359086cc52d9 100644 --- a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java +++ b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java @@ -320,7 +320,7 @@ private static List filesToScan(TableScan tableScan) { Iterables.transform( tableScan.planFiles(), t -> { - String path = t.file().path().toString(); + String path = t.file().location(); return path.split("\\.")[0]; }); return Lists.newArrayList(filesToRead); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java index 01d38dc129c9..6438b794cf53 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java @@ -165,7 +165,7 @@ void checkEntry( void checkDataFile(ContentFile dataFile, FileContent content) { // DataFile is the superclass of DeleteFile, so this method can check both assertThat(dataFile.content()).isEqualTo(content); - assertThat(dataFile.path()).isEqualTo(PATH); + assertThat(dataFile.location()).isEqualTo(PATH); assertThat(dataFile.format()).isEqualTo(FORMAT); assertThat(dataFile.partition()).isEqualTo(PARTITION); assertThat(dataFile.recordCount()).isEqualTo(METRICS.recordCount()); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java index 63c6779298e0..4c74d3f6308d 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java @@ -54,7 +54,7 @@ public void testManifestReaderWithEmptyInheritableMetadata() throws IOException try (ManifestReader reader = ManifestFiles.read(manifest, FILE_IO)) { ManifestEntry entry = Iterables.getOnlyElement(reader.entries()); assertThat(entry.status()).isEqualTo(Status.EXISTING); - assertThat(entry.file().path()).isEqualTo(FILE_A.path()); + assertThat(entry.file().location()).isEqualTo(FILE_A.location()); assertThat(entry.snapshotId()).isEqualTo(1000L); } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java index 175178e48167..4a5554e72d76 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java @@ -154,7 +154,7 @@ public void testReadIteratorWithProjectStats() throws IOException { .project(new Schema(ImmutableList.of(DataFile.FILE_PATH, DataFile.VALUE_COUNTS)))) { DataFile entry = reader.iterator().next(); - assertThat(entry.path()).isEqualTo(FILE_PATH); + assertThat(entry.location()).isEqualTo(FILE_PATH); assertThat(entry.valueCounts()).isEqualTo(VALUE_COUNT); assertThat(entry.columnSizes()).isNull(); assertThat(entry.nullValueCounts()).isNull(); @@ -175,7 +175,7 @@ public void testReadEntriesWithSelectNotProjectStats() throws IOException { DataFile dataFile = entry.file(); // selected field is populated - assertThat(dataFile.path()).isEqualTo(FILE_PATH); + assertThat(dataFile.location()).isEqualTo(FILE_PATH); // not selected fields are all null and not projected assertThat(dataFile.columnSizes()).isNull(); @@ -197,7 +197,7 @@ public void testReadEntriesWithSelectCertainStatNotProjectStats() throws IOExcep DataFile dataFile = reader.iterator().next(); // selected fields are populated - assertThat(dataFile.path()).isEqualTo(FILE_PATH); + assertThat(dataFile.location()).isEqualTo(FILE_PATH); assertThat(dataFile.valueCounts()).isEqualTo(VALUE_COUNT); // not selected fields are all null and not projected @@ -249,7 +249,8 @@ private void assertFullStats(DataFile dataFile) { .isInstanceOf(UnsupportedOperationException.class); } - assertThat(dataFile.path()).isEqualTo(FILE_PATH); // always select file path in all test cases + assertThat(dataFile.location()) + .isEqualTo(FILE_PATH); // always select file path in all test cases } private void assertStatsDropped(DataFile dataFile) { @@ -262,7 +263,8 @@ private void assertStatsDropped(DataFile dataFile) { assertThat(dataFile.lowerBounds()).isNull(); assertThat(dataFile.upperBounds()).isNull(); - assertThat(dataFile.path()).isEqualTo(FILE_PATH); // always select file path in all test cases + assertThat(dataFile.location()) + .isEqualTo(FILE_PATH); // always select file path in all test cases } private void assertNullRecordCount(DataFile dataFile) { diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 9abe7c426f32..0d3cec7d6d55 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -236,7 +236,7 @@ void checkRewrittenEntry( void checkDataFile(ContentFile dataFile, FileContent content) { // DataFile is the superclass of DeleteFile, so this method can check both assertThat(dataFile.content()).isEqualTo(content); - assertThat(dataFile.path()).isEqualTo(PATH); + assertThat(dataFile.location()).isEqualTo(PATH); assertThat(dataFile.format()).isEqualTo(FORMAT); assertThat(dataFile.partition()).isEqualTo(PARTITION); assertThat(dataFile.recordCount()).isEqualTo(METRICS.recordCount()); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index a31e02144167..56b11009fc12 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -1407,12 +1407,12 @@ public void testPositionDeletesWithFilter() { assertThat((Map) constantsMap(posDeleteTask, partitionType)) .as("Expected correct partition spec id on constant column") .containsEntry(MetadataColumns.SPEC_ID.fieldId(), 0); - assertThat(posDeleteTask.file().path()) + assertThat(posDeleteTask.file().location()) .as("Expected correct delete file on task") - .isEqualTo(fileBDeletes().path()); + .isEqualTo(fileBDeletes().location()); assertThat((Map) constantsMap(posDeleteTask, partitionType)) .as("Expected correct delete file on constant column") - .containsEntry(MetadataColumns.FILE_PATH.fieldId(), fileBDeletes().path().toString()); + .containsEntry(MetadataColumns.FILE_PATH.fieldId(), fileBDeletes().location()); } @TestTemplate @@ -1477,12 +1477,12 @@ private void testPositionDeletesBaseTableFilter(boolean transactional) { assertThat((Map) constantsMap(posDeleteTask, partitionType)) .as("Expected correct partition spec id on constant column") .containsEntry(MetadataColumns.SPEC_ID.fieldId(), 0); - assertThat(posDeleteTask.file().path()) + assertThat(posDeleteTask.file().location()) .as("Expected correct delete file on task") - .isEqualTo(fileADeletes().path()); + .isEqualTo(fileADeletes().location()); assertThat((Map) constantsMap(posDeleteTask, partitionType)) .as("Expected correct delete file on constant column") - .containsEntry(MetadataColumns.FILE_PATH.fieldId(), fileADeletes().path().toString()); + .containsEntry(MetadataColumns.FILE_PATH.fieldId(), fileADeletes().location()); } @TestTemplate @@ -1560,7 +1560,7 @@ public void testPositionDeletesWithBaseTableFilterNot() { .as("Expected correct partition spec id on constant column") .containsEntry(MetadataColumns.SPEC_ID.fieldId(), 1); - assertThat(posDeleteTask.file().path()) + assertThat(posDeleteTask.file().location()) .as("Expected correct delete file on task") .isEqualTo(path1); assertThat((Map) constantsMap(posDeleteTask, partitionType)) diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java index 84860d34bb31..03338804d8bc 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java @@ -203,12 +203,12 @@ public void testPositionDeletesPartitionSpecRemoval() { assertThat((Map) constantsMap(posDeleteTask, partitionType)) .as("Expected correct partition spec id on constant column") .containsEntry(MetadataColumns.SPEC_ID.fieldId(), table.ops().current().spec().specId()); - assertThat(posDeleteTask.file().path()) + assertThat(posDeleteTask.file().location()) .as("Expected correct delete file on task") - .isEqualTo(deleteFile.path()); + .isEqualTo(deleteFile.location()); assertThat((Map) constantsMap(posDeleteTask, partitionType)) .as("Expected correct delete file on constant column") - .containsEntry(MetadataColumns.FILE_PATH.fieldId(), deleteFile.path().toString()); + .containsEntry(MetadataColumns.FILE_PATH.fieldId(), deleteFile.location()); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java index 8c6f18619ac0..d6faaba77709 100644 --- a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java +++ b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java @@ -178,7 +178,7 @@ private static List filesToScan(Iterable tasks) { Iterables.transform( tasks, t -> { - String path = t.file().path().toString(); + String path = t.file().location(); return path.split("\\.")[0]; }); return Lists.newArrayList(filesToRead); diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index f95fe6191e43..44bbd069e27d 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -141,7 +141,7 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path() // deleted + FILE_A.location() // deleted )); } @@ -196,7 +196,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path() // deleted + FILE_A.location() // deleted )); } @@ -292,7 +292,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { .findFirst() .get() .path(), // manifest is no longer referenced - FILE_B.path()) // added, but rolled back + FILE_B.location()) // added, but rolled back ); } @@ -652,7 +652,7 @@ public void testScanExpiredManifestInValidSnapshotAppend() { removeSnapshots(table).expireOlderThan(t3).deleteWith(deletedFiles::add).commit(); - assertThat(deletedFiles).contains(FILE_A.path().toString()); + assertThat(deletedFiles).contains(FILE_A.location().toString()); } @TestTemplate @@ -678,7 +678,7 @@ public void testScanExpiredManifestInValidSnapshotFastAppend() { removeSnapshots(table).expireOlderThan(t3).deleteWith(deletedFiles::add).commit(); - assertThat(deletedFiles).contains(FILE_A.path().toString()); + assertThat(deletedFiles).contains(FILE_A.location().toString()); } @TestTemplate @@ -715,8 +715,8 @@ public void dataFilesCleanup() throws IOException { removeSnapshots(table).expireOlderThan(t4).deleteWith(deletedFiles::add).commit(); - assertThat(deletedFiles).contains(FILE_A.path().toString()); - assertThat(deletedFiles).contains(FILE_B.path().toString()); + assertThat(deletedFiles).contains(FILE_A.location().toString()); + assertThat(deletedFiles).contains(FILE_B.location().toString()); } @TestTemplate @@ -789,8 +789,8 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { .containsExactly( "remove-snapshot-3", "remove-snapshot-2", "remove-snapshot-1", "remove-snapshot-0"); - assertThat(deletedFiles).contains(FILE_A.path().toString()); - assertThat(deletedFiles).contains(FILE_B.path().toString()); + assertThat(deletedFiles).contains(FILE_A.location().toString()); + assertThat(deletedFiles).contains(FILE_B.location().toString()); assertThat(planThreadsIndex.get()) .as("Thread should be created in provided pool") .isGreaterThan(0); @@ -857,7 +857,7 @@ public void testWithExpiringDanglingStageCommit() { .addedDataFiles(table.io()) .forEach( i -> { - expectedDeletes.add(i.path().toString()); + expectedDeletes.add(i.location().toString()); }); // ManifestList should be deleted too @@ -923,7 +923,7 @@ public void testWithCherryPickTableSnapshot() { i.addedDataFiles(table.io()) .forEach( item -> { - assertThat(deletedFiles).doesNotContain(item.path().toString()); + assertThat(deletedFiles).doesNotContain(item.location().toString()); }); }); } @@ -969,7 +969,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - assertThat(deletedFiles).doesNotContain(item.path().toString()); + assertThat(deletedFiles).doesNotContain(item.location().toString()); }); }); @@ -986,7 +986,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - assertThat(deletedFiles).doesNotContain(item.path().toString()); + assertThat(deletedFiles).doesNotContain(item.location().toString()); }); }); } @@ -1105,8 +1105,8 @@ public void testExpireWithDeleteFiles() { .as("Should remove old delete files and delete file manifests") .isEqualTo( ImmutableSet.builder() - .add(FILE_A.path()) - .add(FILE_A_DELETES.path()) + .add(FILE_A.location()) + .add(FILE_A_DELETES.location()) .add(firstSnapshot.manifestListLocation()) .add(secondSnapshot.manifestListLocation()) .add(thirdSnapshot.manifestListLocation()) @@ -1614,7 +1614,7 @@ public void testRetainFilesOnRetainedBranches() { expectedDeletes.addAll(manifestPaths(appendA, table.io())); expectedDeletes.add(branchDelete.manifestListLocation()); expectedDeletes.addAll(manifestPaths(branchDelete, table.io())); - expectedDeletes.add(FILE_A.path().toString()); + expectedDeletes.add(FILE_A.location().toString()); assertThat(table.snapshots()).hasSize(2); assertThat(deletedFiles).isEqualTo(expectedDeletes); diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index 72bb85c0446e..e96cd5c0cccf 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -138,7 +138,7 @@ public void testRewriteManifestsGeneratedAndAppendedDirectly() throws IOExceptio List files; List ids; try (ManifestReader reader = ManifestFiles.read(manifests.get(0), table.io())) { - if (reader.iterator().next().path().equals(FILE_A.path())) { + if (reader.iterator().next().location().equals(FILE_A.location())) { files = Arrays.asList(FILE_A, FILE_B); ids = Arrays.asList(manifestAppendId, fileAppendId); } else { @@ -164,7 +164,7 @@ public void testReplaceManifestsSeparate() { // cluster by path will split the manifest into two - table.rewriteManifests().clusterBy(file -> file.path()).commit(); + table.rewriteManifests().clusterBy(file -> file.location()).commit(); List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).hasSize(2); @@ -198,7 +198,7 @@ public void testReplaceManifestsConsolidate() throws IOException { List files; List ids; try (ManifestReader reader = ManifestFiles.read(manifests.get(0), table.io())) { - if (reader.iterator().next().path().equals(FILE_A.path())) { + if (reader.iterator().next().location().equals(FILE_A.location())) { files = Arrays.asList(FILE_A, FILE_B); ids = Arrays.asList(appendIdA, appendIdB); } else { @@ -237,7 +237,7 @@ public void testReplaceManifestsWithFilter() throws IOException { .rewriteIf( manifest -> { try (ManifestReader reader = ManifestFiles.read(manifest, table.io())) { - return !reader.iterator().next().path().equals(FILE_A.path()); + return !reader.iterator().next().location().equals(FILE_A.location()); } catch (IOException x) { throw new RuntimeIOException(x); } @@ -251,7 +251,7 @@ public void testReplaceManifestsWithFilter() throws IOException { List files; List ids; try (ManifestReader reader = ManifestFiles.read(manifests.get(0), table.io())) { - if (reader.iterator().next().path().equals(FILE_B.path())) { + if (reader.iterator().next().location().equals(FILE_B.location())) { files = Arrays.asList(FILE_B, FILE_C); ids = Arrays.asList(appendIdB, appendIdC); } else { @@ -312,7 +312,7 @@ public void testConcurrentRewriteManifest() throws IOException { .rewriteIf( manifest -> { try (ManifestReader reader = ManifestFiles.read(manifest, table.io())) { - return !reader.iterator().next().path().equals(FILE_A.path()); + return !reader.iterator().next().location().equals(FILE_A.location()); } catch (IOException x) { throw new RuntimeIOException(x); } @@ -332,7 +332,7 @@ public void testConcurrentRewriteManifest() throws IOException { List files; List ids; try (ManifestReader reader = ManifestFiles.read(manifests.get(0), table.io())) { - if (reader.iterator().next().path().equals(FILE_A.path())) { + if (reader.iterator().next().location().equals(FILE_A.location())) { files = Arrays.asList(FILE_A, FILE_B); ids = Arrays.asList(appendIdA, appendIdB); } else { @@ -850,7 +850,7 @@ public void testManifestReplacementCombinedWithRewrite() throws IOException { .rewriteIf( manifest -> { try (ManifestReader reader = ManifestFiles.read(manifest, table.io())) { - return !reader.iterator().next().path().equals(FILE_B.path()); + return !reader.iterator().next().location().equals(FILE_B.location()); } catch (IOException x) { throw new RuntimeIOException(x); } @@ -1107,7 +1107,7 @@ public void testRewriteDataManifestsPreservesDeletes() { assertManifestCounts(table, 1, 1); // rewrite manifests and cluster entries by file path - table.rewriteManifests().clusterBy(file -> file.path().toString()).commit(); + table.rewriteManifests().clusterBy(ContentFile::location).commit(); Snapshot rewriteSnapshot = table.currentSnapshot(); diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index b41be0c7a636..0045a7486254 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -137,7 +137,7 @@ public void testValidateDataFilesExistDefaults() { .newRowDelta() .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + .validateDataFilesExist(ImmutableList.of(FILE_A.location())), branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); @@ -153,7 +153,7 @@ public void testValidateDataFilesExistDefaults() { table .newRowDelta() .addDeletes(fileBDeletes()) - .validateDataFilesExist(ImmutableList.of(FILE_B.path())) + .validateDataFilesExist(ImmutableList.of(FILE_B.location())) .validateFromSnapshot(validateFromSnapshotId), branch); @@ -188,7 +188,7 @@ public void testValidateDataFilesExistOverwrite() { .newRowDelta() .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + .validateDataFilesExist(ImmutableList.of(FILE_A.location())), branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); @@ -220,7 +220,7 @@ public void testValidateDataFilesExistReplacePartitions() { .newRowDelta() .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + .validateDataFilesExist(ImmutableList.of(FILE_A.location())), branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); @@ -253,7 +253,7 @@ public void testValidateDataFilesExistFromSnapshot() { .newRowDelta() .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + .validateDataFilesExist(ImmutableList.of(FILE_A.location())), branch); Snapshot snap = latestSnapshot(table, branch); @@ -312,7 +312,7 @@ public void testValidateDataFilesExistRewrite() { .newRowDelta() .addDeletes(fileADeletes()) .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + .validateDataFilesExist(ImmutableList.of(FILE_A.location())), branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); @@ -345,7 +345,7 @@ public void testValidateDataFilesExistValidateDeletes() { .addDeletes(fileADeletes()) .validateDeletedFiles() .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())), + .validateDataFilesExist(ImmutableList.of(FILE_A.location())), branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); @@ -411,7 +411,7 @@ public void testValidateNoConflictsFromSnapshot() { .addDeletes(fileADeletes()) .validateDeletedFiles() .validateFromSnapshot(validateFromSnapshotId) - .validateDataFilesExist(ImmutableList.of(FILE_A.path())) + .validateDataFilesExist(ImmutableList.of(FILE_A.location())) .conflictDetectionFilter(Expressions.equal("data", "u")) // bucket16("u") -> 0 .validateNoConflictingDataFiles(), branch); @@ -744,7 +744,7 @@ public void testValidateDataFilesExistWithConflictDetectionFilter() { table .newRowDelta() .addDeletes(deleteFile) - .validateDataFilesExist(ImmutableList.of(dataFile1.path())) + .validateDataFilesExist(ImmutableList.of(dataFile1.location())) .validateDeletedFiles() .validateFromSnapshot(baseSnapshot.snapshotId()) .conflictDetectionFilter(conflictDetectionFilter) @@ -798,7 +798,7 @@ public void testValidateDataFilesDoNotExistWithConflictDetectionFilter() { table .newRowDelta() .addDeletes(deleteFile) - .validateDataFilesExist(ImmutableList.of(dataFile1.path())) + .validateDataFilesExist(ImmutableList.of(dataFile1.location())) .validateDeletedFiles() .validateFromSnapshot(baseSnapshot.snapshotId()) .conflictDetectionFilter(conflictDetectionFilter) @@ -1035,7 +1035,7 @@ public void testAbortMultipleSpecs() { .addDeletes(secondDeleteFile) .deleteWith(deletedFiles::add) .validateDeletedFiles() - .validateDataFilesExist(ImmutableList.of(firstSnapshotDataFile.path())); + .validateDataFilesExist(ImmutableList.of(firstSnapshotDataFile.location())); rowDelta.apply(); diff --git a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java index 7845f305e3c9..d416a25d1c72 100644 --- a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java +++ b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java @@ -73,7 +73,7 @@ public void testRewrite() { // FILE_A and FILE_B in manifest may reorder for (ManifestEntry entry : ManifestFiles.read(newManifest, FILE_IO).entries()) { - if (entry.file().path().equals(FILE_A.path())) { + if (entry.file().location().equals(FILE_A.location())) { V2Assert.assertEquals( "FILE_A sequence number should be 1", 1, entry.dataSequenceNumber().longValue()); V2Assert.assertEquals( @@ -86,7 +86,7 @@ public void testRewrite() { entry.file().fileSequenceNumber().longValue()); } - if (entry.file().path().equals(FILE_B.path())) { + if (entry.file().location().equals(FILE_B.location())) { V2Assert.assertEquals( "FILE_b sequence number should be 2", 2, entry.dataSequenceNumber().longValue()); V2Assert.assertEquals( diff --git a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java index 80f4d059f5f0..efd910bd7eef 100644 --- a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java +++ b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java @@ -84,12 +84,13 @@ public void testPartitionedTableWithPartitionEqDeletes() { verifyManifestSequenceNumber(deleteManifest, 1, 1); assertThat(table.newScan().planFiles()) .hasSize(3) - .filteredOn(fileScanTask -> fileScanTask.file().path().equals(FILE_A.path())) + .filteredOn(fileScanTask -> fileScanTask.file().location().equals(FILE_A.location())) .first() .satisfies( fileScanTask -> { assertThat(fileScanTask.deletes()).hasSize(1); - assertThat(fileScanTask.deletes().get(0).path()).isEqualTo(FILE_A_EQ_1.path()); + assertThat(fileScanTask.deletes().get(0).location()) + .isEqualTo(FILE_A_EQ_1.location()); }); // first commit after row-delta changes @@ -103,7 +104,7 @@ public void testPartitionedTableWithPartitionEqDeletes() { verifyManifestSequenceNumber(dataManifest2, 2, 0); assertThat(table.newScan().planFiles()) .hasSize(2) - .filteredOn(fileScanTask -> fileScanTask.file().path().equals(FILE_A.path())) + .filteredOn(fileScanTask -> fileScanTask.file().location().equals(FILE_A.location())) .first() .satisfies(fileScanTask -> assertThat(fileScanTask.deletes()).hasSize(1)); @@ -117,7 +118,7 @@ public void testPartitionedTableWithPartitionEqDeletes() { verifyManifestSequenceNumber(dataManifests.get(0), 3, 0); assertThat(table.newScan().planFiles()) .hasSize(1) - .filteredOn(fileScanTask -> fileScanTask.file().path().equals(FILE_A.path())) + .filteredOn(fileScanTask -> fileScanTask.file().location().equals(FILE_A.location())) .first() .satisfies(fileScanTask -> assertThat(fileScanTask.deletes()).hasSize(1)); } @@ -138,7 +139,7 @@ public void testPartitionedTableWithUnrelatedPartitionDeletes() { .first() .satisfies( fileScanTask -> { - assertThat(fileScanTask.file().path()).isEqualTo(FILE_B.path()); + assertThat(fileScanTask.file().location()).isEqualTo(FILE_B.location()); assertThat(fileScanTask.deletes()).isEmpty(); }); @@ -199,10 +200,10 @@ public void testPartitionedTableWithExistingDeleteFile() { assertThat(tasks).hasSize(1); FileScanTask task = tasks.get(0); - assertThat(task.file().path()).isEqualTo(FILE_A.path()); + assertThat(task.file().location()).isEqualTo(FILE_A.location()); assertThat(task.deletes()).hasSize(2); - assertThat(task.deletes().get(0).path()).isEqualTo(FILE_A_EQ_1.path()); - assertThat(task.deletes().get(1).path()).isEqualTo(FILE_A_POS_1.path()); + assertThat(task.deletes().get(0).location()).isEqualTo(FILE_A_EQ_1.location()); + assertThat(task.deletes().get(1).location()).isEqualTo(FILE_A_POS_1.location()); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java index 5402a13d7d4b..75983402a38c 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -2744,7 +2744,7 @@ public void assertFilePartitionSpec(Table table, DataFile dataFile, int specId) try (CloseableIterable tasks = table.newScan().planFiles()) { Streams.stream(tasks) .map(FileScanTask::file) - .filter(file -> file.path().equals(dataFile.path())) + .filter(file -> file.location().equals(dataFile.location())) .forEach(file -> assertThat(file.specId()).as("Spec ID should match").isEqualTo(specId)); } catch (IOException e) { throw new UncheckedIOException(e); diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java b/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java index 79f30e109f7d..6765b23d8ab8 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java @@ -30,6 +30,7 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.GenericBlobMetadata; @@ -220,7 +221,7 @@ private static Set manifestLocations(Set snapshotSet, FileIO i private static Set dataLocations(Set snapshotSet, FileIO io) { return snapshotSet.stream() .flatMap(snapshot -> StreamSupport.stream(snapshot.addedDataFiles(io).spliterator(), false)) - .map(dataFile -> dataFile.path().toString()) + .map(ContentFile::location) .collect(Collectors.toSet()); } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java index fa4b227a1a9d..a24c4245b157 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java @@ -194,13 +194,13 @@ private static Set getFiles(Table table) throws IOException { .equals(MetadataTableType.POSITION_DELETES))) { try (CloseableIterable tasks = table.newBatchScan().planFiles()) { for (ScanTask task : tasks) { - files.add(((PositionDeletesScanTask) task).file().path()); + files.add(((PositionDeletesScanTask) task).file().location()); } } } else { try (CloseableIterable tasks = table.newScan().planFiles()) { for (FileScanTask task : tasks) { - files.add(task.file().path()); + files.add(task.file().location()); } } } From 821aec32bb9be28d9c1905f772d9e3101cc98d9e Mon Sep 17 00:00:00 2001 From: Aihua Xu Date: Fri, 15 Nov 2024 10:52:23 -0800 Subject: [PATCH 145/313] API: Add Variant data type (#11324) --- .../main/java/org/apache/iceberg/Schema.java | 2 +- .../iceberg/expressions/ExpressionUtil.java | 5 +- .../apache/iceberg/transforms/Identity.java | 3 + .../apache/iceberg/types/FindTypeVisitor.java | 9 ++ .../org/apache/iceberg/types/IndexByName.java | 5 + .../apache/iceberg/types/IndexParents.java | 5 + .../java/org/apache/iceberg/types/Type.java | 3 +- .../org/apache/iceberg/types/TypeUtil.java | 8 ++ .../java/org/apache/iceberg/types/Types.java | 35 ++++++ .../iceberg/TestPartitionSpecValidation.java | 15 ++- .../java/org/apache/iceberg/TestSchema.java | 5 +- .../iceberg/transforms/TestBucketing.java | 14 +++ .../iceberg/transforms/TestIdentity.java | 18 ++++ .../iceberg/types/TestSerializableTypes.java | 11 +- .../org/apache/iceberg/TableMetadata.java | 5 +- .../org/apache/iceberg/TestSortOrder.java | 16 +++ .../org/apache/iceberg/TestTableMetadata.java | 100 ++++++++++-------- 17 files changed, 208 insertions(+), 51 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index a94e8771875a..bd07e9798e9b 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -60,7 +60,7 @@ public class Schema implements Serializable { @VisibleForTesting static final Map MIN_FORMAT_VERSIONS = - ImmutableMap.of(Type.TypeID.TIMESTAMP_NANO, 3); + ImmutableMap.of(Type.TypeID.TIMESTAMP_NANO, 3, Type.TypeID.VARIANT, 3); private final StructType struct; private final int schemaId; diff --git a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java index 82d513ced7dd..9be144b2ce98 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -534,7 +534,8 @@ private static String sanitize(Type type, Object value, long now, int today) { case DECIMAL: case FIXED: case BINARY: - // for boolean, uuid, decimal, fixed, and binary, match the string result + case VARIANT: + // for boolean, uuid, decimal, fixed, variant, and binary, match the string result return sanitizeSimpleString(value.toString()); } throw new UnsupportedOperationException( @@ -562,7 +563,7 @@ private static String sanitize(Literal literal, long now, int today) { } else if (literal instanceof Literals.DoubleLiteral) { return sanitizeNumber(((Literals.DoubleLiteral) literal).value(), "float"); } else { - // for uuid, decimal, fixed, and binary, match the string result + // for uuid, decimal, fixed, variant, and binary, match the string result return sanitizeSimpleString(literal.value().toString()); } } diff --git a/api/src/main/java/org/apache/iceberg/transforms/Identity.java b/api/src/main/java/org/apache/iceberg/transforms/Identity.java index d4e5e532943b..04f0c25e9222 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Identity.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Identity.java @@ -38,6 +38,9 @@ class Identity implements Transform { */ @Deprecated public static Identity get(Type type) { + Preconditions.checkArgument( + type.typeId() != Type.TypeID.VARIANT, "Unsupported type for identity: %s", type); + return new Identity<>(type); } diff --git a/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java b/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java index d1dc4adc214a..f0750f337e2e 100644 --- a/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java +++ b/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java @@ -76,6 +76,15 @@ public Type map(Types.MapType map, Type keyResult, Type valueResult) { return valueResult; } + @Override + public Type variant() { + if (predicate.test(Types.VariantType.get())) { + return Types.VariantType.get(); + } + + return null; + } + @Override public Type primitive(Type.PrimitiveType primitive) { if (predicate.test(primitive)) { diff --git a/api/src/main/java/org/apache/iceberg/types/IndexByName.java b/api/src/main/java/org/apache/iceberg/types/IndexByName.java index 9183ea85f467..131434c9a156 100644 --- a/api/src/main/java/org/apache/iceberg/types/IndexByName.java +++ b/api/src/main/java/org/apache/iceberg/types/IndexByName.java @@ -176,6 +176,11 @@ public Map map( return nameToId; } + @Override + public Map variant() { + return nameToId; + } + @Override public Map primitive(Type.PrimitiveType primitive) { return nameToId; diff --git a/api/src/main/java/org/apache/iceberg/types/IndexParents.java b/api/src/main/java/org/apache/iceberg/types/IndexParents.java index bcd1e1ee900c..952447ed2799 100644 --- a/api/src/main/java/org/apache/iceberg/types/IndexParents.java +++ b/api/src/main/java/org/apache/iceberg/types/IndexParents.java @@ -76,6 +76,11 @@ public Map map( return idToParent; } + @Override + public Map variant() { + return idToParent; + } + @Override public Map primitive(Type.PrimitiveType primitive) { return idToParent; diff --git a/api/src/main/java/org/apache/iceberg/types/Type.java b/api/src/main/java/org/apache/iceberg/types/Type.java index 571bf9a14e43..30870535521f 100644 --- a/api/src/main/java/org/apache/iceberg/types/Type.java +++ b/api/src/main/java/org/apache/iceberg/types/Type.java @@ -45,7 +45,8 @@ enum TypeID { DECIMAL(BigDecimal.class), STRUCT(StructLike.class), LIST(List.class), - MAP(Map.class); + MAP(Map.class), + VARIANT(Object.class); private final Class javaClass; diff --git a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java index 8a9184569aec..7fcf3db3a40d 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -534,6 +534,7 @@ private static int estimateSize(Type type) { case FIXED: return ((Types.FixedType) type).length(); case BINARY: + case VARIANT: return 80; case DECIMAL: // 12 (header) + (12 + 12 + 4) (BigInteger) + 4 (scale) = 44 bytes @@ -612,6 +613,10 @@ public T map(Types.MapType map, T keyResult, T valueResult) { return null; } + public T variant() { + return null; + } + public T primitive(Type.PrimitiveType primitive) { return null; } @@ -675,6 +680,9 @@ public static T visit(Type type, SchemaVisitor visitor) { return visitor.map(map, keyResult, valueResult); + case VARIANT: + return visitor.variant(); + default: return visitor.primitive(type.asPrimitiveType()); } diff --git a/api/src/main/java/org/apache/iceberg/types/Types.java b/api/src/main/java/org/apache/iceberg/types/Types.java index 4bb1674f3be5..927b3a5065ad 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -412,6 +412,41 @@ public String toString() { } } + public static class VariantType implements Type { + private static final VariantType INSTANCE = new VariantType(); + + public static VariantType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.VARIANT; + } + + @Override + public String toString() { + return "variant"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof VariantType)) { + return false; + } + + VariantType that = (VariantType) o; + return typeId() == that.typeId(); + } + + @Override + public int hashCode() { + return Objects.hash(VariantType.class, typeId()); + } + } + public static class DecimalType extends PrimitiveType { public static DecimalType of(int precision, int scale) { return new DecimalType(precision, scale); diff --git a/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java b/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java index 263db427aa16..971f5a9e4510 100644 --- a/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java +++ b/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; @@ -34,7 +35,8 @@ public class TestPartitionSpecValidation { NestedField.required(3, "another_ts", Types.TimestampType.withZone()), NestedField.required(4, "d", Types.TimestampType.withZone()), NestedField.required(5, "another_d", Types.TimestampType.withZone()), - NestedField.required(6, "s", Types.StringType.get())); + NestedField.required(6, "s", Types.StringType.get()), + NestedField.required(7, "v", Types.VariantType.get())); @Test public void testMultipleTimestampPartitions() { @@ -312,4 +314,15 @@ public void testAddPartitionFieldsWithAndWithoutFieldIds() { assertThat(spec.fields().get(2).fieldId()).isEqualTo(1006); assertThat(spec.lastAssignedFieldId()).isEqualTo(1006); } + + @Test + public void testVariantUnsupported() { + assertThatThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA) + .add(7, 1005, "variant_partition1", Transforms.bucket(5)) + .build()) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot partition by non-primitive source field: variant"); + } } diff --git a/api/src/test/java/org/apache/iceberg/TestSchema.java b/api/src/test/java/org/apache/iceberg/TestSchema.java index e79adbd09fb7..e9cb387eebb5 100644 --- a/api/src/test/java/org/apache/iceberg/TestSchema.java +++ b/api/src/test/java/org/apache/iceberg/TestSchema.java @@ -38,7 +38,10 @@ public class TestSchema { private static final List TEST_TYPES = - ImmutableList.of(Types.TimestampNanoType.withoutZone(), Types.TimestampNanoType.withZone()); + ImmutableList.of( + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), + Types.VariantType.get()); private static final Schema INITIAL_DEFAULT_SCHEMA = new Schema( diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java index fc4333d7c6c5..5f0cac2b5e8c 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java @@ -417,6 +417,20 @@ public void testVerifiedIllegalNumBuckets() { .hasMessage("Invalid number of buckets: 0 (must be > 0)"); } + @Test + public void testVariantUnsupported() { + assertThatThrownBy(() -> Transforms.bucket(Types.VariantType.get(), 3)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot bucket by type: variant"); + + Transform bucket = Transforms.bucket(3); + assertThatThrownBy(() -> bucket.bind(Types.VariantType.get())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot bucket by type: variant"); + + assertThat(bucket.canTransform(Types.VariantType.get())).isFalse(); + } + private byte[] randomBytes(int length) { byte[] bytes = new byte[length]; testRandom.nextBytes(bytes); diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java index 93d3281411f3..b5076e08a947 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -155,4 +156,21 @@ public void testBigDecimalToHumanString() { .as("Should not modify Strings") .isEqualTo(decimalString); } + + @Test + public void testVariantUnsupported() { + assertThatThrownBy(() -> Transforms.identity().bind(Types.VariantType.get())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot bind to unsupported type: variant"); + + assertThatThrownBy(() -> Transforms.fromString(Types.VariantType.get(), "identity")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported type for identity: variant"); + + assertThatThrownBy(() -> Transforms.identity(Types.VariantType.get())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported type for identity: variant"); + + assertThat(Transforms.identity().canTransform(Types.VariantType.get())).isFalse(); + } } diff --git a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java index 97bb2b1a19e4..af2ebae7e1a8 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java @@ -45,7 +45,7 @@ public void testIdentityTypes() throws Exception { Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), - Types.BinaryType.get() + Types.BinaryType.get(), }; for (Type type : identityPrimitives) { @@ -127,6 +127,15 @@ public void testLists() throws Exception { } } + @Test + public void testVariant() throws Exception { + Types.VariantType variant = Types.VariantType.get(); + Type copy = TestHelpers.roundTripSerialize(variant); + assertThat(copy) + .as("Variant serialization should be equal to starting type") + .isEqualTo(variant); + } + @Test public void testSchema() throws Exception { Schema schema = diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 3cdc53995dce..0e323bca1c97 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -295,8 +295,9 @@ public String toString() { sortOrders != null && !sortOrders.isEmpty(), "Sort orders cannot be null or empty"); Preconditions.checkArgument( formatVersion <= SUPPORTED_TABLE_FORMAT_VERSION, - "Unsupported format version: v%s", - formatVersion); + "Unsupported format version: v%s (supported: v%s)", + formatVersion, + SUPPORTED_TABLE_FORMAT_VERSION); Preconditions.checkArgument( formatVersion == 1 || uuid != null, "UUID is required in format v%s", formatVersion); Preconditions.checkArgument( diff --git a/core/src/test/java/org/apache/iceberg/TestSortOrder.java b/core/src/test/java/org/apache/iceberg/TestSortOrder.java index 6e99af3e260e..3d139543b71c 100644 --- a/core/src/test/java/org/apache/iceberg/TestSortOrder.java +++ b/core/src/test/java/org/apache/iceberg/TestSortOrder.java @@ -326,6 +326,22 @@ public void testSortedColumnNames() { assertThat(sortedCols).containsExactly("s.id", "data"); } + @TestTemplate + public void testVariantUnsupported() { + Schema v3Schema = + new Schema( + Types.NestedField.required(3, "id", Types.LongType.get()), + Types.NestedField.required(4, "data", Types.StringType.get()), + Types.NestedField.required( + 5, + "struct", + Types.StructType.of(Types.NestedField.optional(6, "v", Types.VariantType.get())))); + + assertThatThrownBy(() -> SortOrder.builderFor(v3Schema).withOrderId(10).asc("struct.v").build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported type for identity: variant"); + } + @TestTemplate public void testPreservingOrderSortedColumnNames() { SortOrder order = diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 71254b3abb1b..c9a8eb75a986 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -885,7 +885,8 @@ public void testV2UUIDValidation() { @Test public void testVersionValidation() { - int unsupportedVersion = TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1; + int supportedVersion = TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION; + int unsupportedVersion = supportedVersion + 1; assertThatThrownBy( () -> new TableMetadata( @@ -914,7 +915,62 @@ public void testVersionValidation() { ImmutableList.of(), ImmutableList.of())) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Unsupported format version: v" + unsupportedVersion); + .hasMessage( + "Unsupported format version: v%s (supported: v%s)", + unsupportedVersion, supportedVersion); + + assertThatThrownBy( + () -> + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + unsupportedVersion)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Unsupported format version: v%s (supported: v%s)", + unsupportedVersion, supportedVersion); + + // should be allowed in the supported version + assertThat( + new TableMetadata( + null, + supportedVersion, + UUID.randomUUID().toString(), + TEST_LOCATION, + SEQ_NO, + System.currentTimeMillis(), + LAST_ASSIGNED_COLUMN_ID, + 7, + ImmutableList.of(TEST_SCHEMA), + SPEC_5.specId(), + ImmutableList.of(SPEC_5), + SPEC_5.lastAssignedFieldId(), + 3, + ImmutableList.of(SORT_ORDER_3), + ImmutableMap.of(), + -1L, + ImmutableList.of(), + null, + ImmutableList.of(), + ImmutableList.of(), + ImmutableMap.of(), + ImmutableList.of(), + ImmutableList.of(), + ImmutableList.of())) + .isNotNull(); + + assertThat( + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + supportedVersion)) + .isNotNull(); } @Test @@ -1647,46 +1703,6 @@ public void testConstructV3Metadata() { 3); } - @Test - public void testV3TimestampNanoTypeSupport() { - Schema v3Schema = - new Schema( - Types.NestedField.required(3, "id", Types.LongType.get()), - Types.NestedField.required(4, "data", Types.StringType.get()), - Types.NestedField.required( - 5, - "struct", - Types.StructType.of( - Types.NestedField.optional( - 6, "ts_nanos", Types.TimestampNanoType.withZone())))); - - for (int unsupportedFormatVersion : ImmutableList.of(1, 2)) { - assertThatThrownBy( - () -> - TableMetadata.newTableMetadata( - v3Schema, - PartitionSpec.unpartitioned(), - SortOrder.unsorted(), - TEST_LOCATION, - ImmutableMap.of(), - unsupportedFormatVersion)) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Invalid schema for v%s:\n" - + "- Invalid type for struct.ts_nanos: timestamptz_ns is not supported until v3", - unsupportedFormatVersion); - } - - // should be allowed in v3 - TableMetadata.newTableMetadata( - v3Schema, - PartitionSpec.unpartitioned(), - SortOrder.unsorted(), - TEST_LOCATION, - ImmutableMap.of(), - 3); - } - @Test public void onlyMetadataLocationIsUpdatedWithoutTimestampAndMetadataLogEntry() { String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; From 315e1543743c7e72e5d66f0604f99414357ad772 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 15 Nov 2024 22:12:45 +0100 Subject: [PATCH 146/313] Spark 3.5: Adapt DeleteFileIndexBenchmark for DVs (#11529) --- .../iceberg/DeleteFileIndexBenchmark.java | 29 +++++++++++++++---- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java index 365dff804c75..8b0b05911f66 100644 --- a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java +++ b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java @@ -81,8 +81,8 @@ public class DeleteFileIndexBenchmark { private List dataFiles; - @Param({"true", "false"}) - private boolean oneToOneMapping; + @Param({"partition", "file", "dv"}) + private String type; @Setup public void setupBenchmark() throws NoSuchTableException, ParseException { @@ -93,10 +93,12 @@ public void setupBenchmark() throws NoSuchTableException, ParseException { } private void initDataAndDeletes() { - if (oneToOneMapping) { + if (type.equals("partition")) { + initDataAndPartitionScopedDeletes(); + } else if (type.equals("file")) { initDataAndFileScopedDeletes(); } else { - initDataAndPartitionScopedDeletes(); + initDataAndDVs(); } } @@ -183,6 +185,23 @@ private void initDataAndFileScopedDeletes() { } } + private void initDataAndDVs() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + DeleteFile dv = FileGenerationUtil.generateDV(table, dataFile); + rowDelta.addRows(dataFile); + rowDelta.addDeletes(dv); + } + + rowDelta.commit(); + } + } + private void setupSpark() { this.spark = SparkSession.builder() @@ -240,7 +259,7 @@ private void initTable() throws NoSuchTableException, ParseException { TableProperties.DELETE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName(), TableProperties.FORMAT_VERSION, - 2); + type.equals("dv") ? 3 : 2); this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); } From 7e4fd1ba6e4479376128312572ed41378b2ed9a5 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 15 Nov 2024 22:14:32 +0100 Subject: [PATCH 147/313] Spark 3.5: Adapt PlanningBenchmark for DVs (#11531) --- .../iceberg/spark/PlanningBenchmark.java | 58 ++++++++++++++++++- 1 file changed, 57 insertions(+), 1 deletion(-) diff --git a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java index ed97e6b08414..1d51350487c4 100644 --- a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java +++ b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java @@ -60,6 +60,7 @@ import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Measurement; import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.Setup; import org.openjdk.jmh.annotations.State; @@ -107,6 +108,9 @@ public class PlanningBenchmark { private SparkSession spark; private Table table; + @Param({"partition", "file", "dv"}) + private String type; + @Setup public void setupBenchmark() throws NoSuchTableException, ParseException { setupSpark(); @@ -266,7 +270,7 @@ private void initTable() throws NoSuchTableException, ParseException { TableProperties.DELETE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName(), TableProperties.FORMAT_VERSION, - 2); + type.equals("dv") ? 3 : 2); this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); } @@ -276,6 +280,16 @@ private void dropTable() { } private void initDataAndDeletes() { + if (type.equals("partition")) { + initDataAndPartitionScopedDeletes(); + } else if (type.equals("file")) { + initDataAndFileScopedDeletes(); + } else { + initDataAndDVs(); + } + } + + private void initDataAndPartitionScopedDeletes() { for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { StructLike partition = TestHelpers.Row.of(partitionOrdinal); @@ -299,6 +313,48 @@ private void initDataAndDeletes() { } } + private void initDataAndFileScopedDeletes() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = generateDataFile(partition, Integer.MIN_VALUE, Integer.MIN_VALUE); + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, dataFile); + rowDelta.addRows(dataFile); + rowDelta.addDeletes(deleteFile); + } + + // add one data file that would match the sort key predicate + DataFile sortKeyDataFile = generateDataFile(partition, SORT_KEY_VALUE, SORT_KEY_VALUE); + rowDelta.addRows(sortKeyDataFile); + + rowDelta.commit(); + } + } + + private void initDataAndDVs() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = generateDataFile(partition, Integer.MIN_VALUE, Integer.MIN_VALUE); + DeleteFile dv = FileGenerationUtil.generateDV(table, dataFile); + rowDelta.addRows(dataFile); + rowDelta.addDeletes(dv); + } + + // add one data file that would match the sort key predicate + DataFile sortKeyDataFile = generateDataFile(partition, SORT_KEY_VALUE, SORT_KEY_VALUE); + rowDelta.addRows(sortKeyDataFile); + + rowDelta.commit(); + } + } + private DataFile generateDataFile(StructLike partition, int sortKeyMin, int sortKeyMax) { int sortKeyFieldId = table.schema().findField(SORT_KEY_COLUMN).fieldId(); ByteBuffer lower = Conversions.toByteBuffer(Types.IntegerType.get(), sortKeyMin); From acd7cc1126b192ccb53ad8198bda37e983aa4c6c Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 15 Nov 2024 22:15:08 +0100 Subject: [PATCH 148/313] Spark 3.5: Add DVReaderBenchmark (#11537) --- .../spark/source/DVReaderBenchmark.java | 267 ++++++++++++++++++ 1 file changed, 267 insertions(+) create mode 100644 spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java new file mode 100644 index 000000000000..c6794e43c636 --- /dev/null +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.io.DeleteWriteResult; +import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.unsafe.types.UTF8String; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that compares the performance of DV and position delete readers. + * + *

    To run this benchmark for spark-3.5: + * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5_2.12:jmh + * -PjmhIncludeRegex=DVReaderBenchmark + * -PjmhOutputPath=benchmark/iceberg-dv-reader-benchmark-result.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 15) +@Timeout(time = 20, timeUnit = TimeUnit.MINUTES) +@BenchmarkMode(Mode.SingleShotTime) +public class DVReaderBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final int DATA_FILE_RECORD_COUNT = 2_000_000; + private static final long TARGET_FILE_SIZE = Long.MAX_VALUE; + + @Param({"5", "10"}) + private int referencedDataFileCount; + + @Param({"0.01", "0.03", "0.05", "0.10", "0.2"}) + private double deletedRowsRatio; + + private final Configuration hadoopConf = new Configuration(); + private final Random random = ThreadLocalRandom.current(); + private SparkSession spark; + private Table table; + private DeleteWriteResult dvsResult; + private DeleteWriteResult fileDeletesResult; + private DeleteWriteResult partitionDeletesResult; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException, IOException { + setupSpark(); + initTable(); + List deletes = generatePositionDeletes(); + this.dvsResult = writeDVs(deletes); + this.fileDeletesResult = writePositionDeletes(deletes, DeleteGranularity.FILE); + this.partitionDeletesResult = writePositionDeletes(deletes, DeleteGranularity.PARTITION); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + tearDownSpark(); + } + + @Benchmark + @Threads(1) + public void dv(Blackhole blackhole) { + DeleteLoader loader = new BaseDeleteLoader(file -> table.io().newInputFile(file), null); + DeleteFile dv = dvsResult.deleteFiles().get(0); + CharSequence dataFile = dv.referencedDataFile(); + PositionDeleteIndex index = loader.loadPositionDeletes(ImmutableList.of(dv), dataFile); + blackhole.consume(index); + } + + @Benchmark + @Threads(1) + public void fileScopedParquetDeletes(Blackhole blackhole) { + DeleteLoader loader = new BaseDeleteLoader(file -> table.io().newInputFile(file), null); + DeleteFile deleteFile = fileDeletesResult.deleteFiles().get(0); + CharSequence dataFile = ContentFileUtil.referencedDataFile(deleteFile); + PositionDeleteIndex index = loader.loadPositionDeletes(ImmutableList.of(deleteFile), dataFile); + blackhole.consume(index); + } + + @Benchmark + @Threads(1) + public void partitionScopedParquetDeletes(Blackhole blackhole) { + DeleteLoader loader = new BaseDeleteLoader(file -> table.io().newInputFile(file), null); + DeleteFile deleteFile = Iterables.getOnlyElement(partitionDeletesResult.deleteFiles()); + CharSequence dataFile = Iterables.getLast(partitionDeletesResult.referencedDataFiles()); + PositionDeleteIndex index = loader.loadPositionDeletes(ImmutableList.of(deleteFile), dataFile); + blackhole.consume(index); + } + + private FanoutPositionOnlyDeleteWriter newWriter(DeleteGranularity granularity) { + return new FanoutPositionOnlyDeleteWriter<>( + newWriterFactory(), + newFileFactory(FileFormat.PARQUET), + table.io(), + TARGET_FILE_SIZE, + granularity); + } + + private SparkFileWriterFactory newWriterFactory() { + return SparkFileWriterFactory.builderFor(table).dataFileFormat(FileFormat.PARQUET).build(); + } + + private OutputFileFactory newFileFactory(FileFormat format) { + return OutputFileFactory.builderFor(table, 1, 1).format(format).build(); + } + + private List generatePositionDeletes() { + int numDeletesPerFile = (int) (DATA_FILE_RECORD_COUNT * deletedRowsRatio); + int numDeletes = referencedDataFileCount * numDeletesPerFile; + List deletes = Lists.newArrayListWithExpectedSize(numDeletes); + + for (int pathIndex = 0; pathIndex < referencedDataFileCount; pathIndex++) { + UTF8String dataFilePath = UTF8String.fromString(generateDataFilePath()); + Set positions = generatePositions(numDeletesPerFile); + for (long pos : positions) { + deletes.add(new GenericInternalRow(new Object[] {dataFilePath, pos})); + } + } + + Collections.shuffle(deletes); + + return deletes; + } + + private DeleteWriteResult writeDVs(Iterable rows) throws IOException { + OutputFileFactory fileFactory = newFileFactory(FileFormat.PUFFIN); + DVFileWriter writer = new BaseDVFileWriter(fileFactory, path -> null); + try (DVFileWriter closableWriter = writer) { + for (InternalRow row : rows) { + String path = row.getString(0); + long pos = row.getLong(1); + closableWriter.delete(path, pos, table.spec(), null); + } + } + return writer.result(); + } + + private DeleteWriteResult writePositionDeletes( + Iterable rows, DeleteGranularity granularity) throws IOException { + FanoutPositionOnlyDeleteWriter writer = newWriter(granularity); + try (FanoutPositionOnlyDeleteWriter closableWriter = writer) { + PositionDelete positionDelete = PositionDelete.create(); + for (InternalRow row : rows) { + String path = row.getString(0); + long pos = row.getLong(1); + positionDelete.set(path, pos, null /* no row */); + closableWriter.write(positionDelete, table.spec(), null); + } + } + return writer.result(); + } + + public Set generatePositions(int numPositions) { + Set positions = Sets.newHashSet(); + + while (positions.size() < numPositions) { + long pos = random.nextInt(DATA_FILE_RECORD_COUNT); + positions.add(pos); + } + + return positions; + } + + private String generateDataFilePath() { + String fileName = FileGenerationUtil.generateFileName(); + return table.locationProvider().newDataLocation(table.spec(), null, fileName); + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .master("local[*]") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (c1 INT, c2 INT, c3 STRING) USING iceberg", TABLE_NAME); + this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} From 8c83fb73402d8ef1e3f8f673a1851d1d244fcc10 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 17 Nov 2024 08:29:05 +0100 Subject: [PATCH 149/313] Build: Bump datamodel-code-generator from 0.26.2 to 0.26.3 (#11572) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.26.2 to 0.26.3. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.26.2...0.26.3) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index ba58048e02f7..15cb9d7dbe4e 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.1 -datamodel-code-generator==0.26.2 +datamodel-code-generator==0.26.3 From 3934b1383db31aaeb75009f1fd725ca414a78381 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 17 Nov 2024 08:29:46 +0100 Subject: [PATCH 150/313] Build: Bump software.amazon.awssdk:bom from 2.29.9 to 2.29.15 (#11568) Bumps software.amazon.awssdk:bom from 2.29.9 to 2.29.15. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 54a9f14b4b42..6203bece86b6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.29.9" +awssdk-bom = "2.29.15" azuresdk-bom = "1.2.29" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" From fc11dc456e87e6dcec77d6de9579cf85a6f8fa4c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 17 Nov 2024 08:29:56 +0100 Subject: [PATCH 151/313] Build: Bump io.netty:netty-buffer from 4.1.114.Final to 4.1.115.Final (#11569) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.114.Final to 4.1.115.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.114.Final...netty-4.1.115.Final) --- updated-dependencies: - dependency-name: io.netty:netty-buffer dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 6203bece86b6..96f62fb8163b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -71,8 +71,8 @@ microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.99.0" -netty-buffer = "4.1.114.Final" -netty-buffer-compat = "4.1.114.Final" +netty-buffer = "4.1.115.Final" +netty-buffer-compat = "4.1.115.Final" object-client-bundle = "3.3.2" orc = "1.9.4" parquet = "1.13.1" From b38951db6a7061a595605229c21c1a1912a3a4c1 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sun, 17 Nov 2024 17:20:20 +0100 Subject: [PATCH 152/313] Data, Flink, MR, Spark: Test deletes with format-version=3 (#11538) --- .../iceberg/TestMetadataTableFilters.java | 14 ++---- .../apache/iceberg/data/DeleteReadTests.java | 32 +++++++++----- .../data/TestGenericReaderDeletes.java | 2 +- .../source/TestFlinkReaderDeletesBase.java | 2 +- .../source/TestFlinkReaderDeletesBase.java | 2 +- .../source/TestFlinkReaderDeletesBase.java | 2 +- .../mr/TestInputFormatReaderDeletes.java | 20 +++++---- .../spark/source/TestSparkReaderDeletes.java | 43 +++++++++++++------ .../spark/source/TestSparkReaderDeletes.java | 34 +++++++++------ .../spark/source/TestSparkReaderDeletes.java | 36 ++++++++++------ 10 files changed, 116 insertions(+), 71 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index 7c5a860db15f..0762d3b2dca4 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -396,8 +396,8 @@ public void testPartitionSpecEvolutionRemovalV2() { .withPartitionPath("id=11") .build(); - DeleteFile delete10 = posDelete(table, data10); - DeleteFile delete11 = posDelete(table, data11); + DeleteFile delete10 = newDeletes(data10); + DeleteFile delete11 = newDeletes(data11); table.newFastAppend().appendFile(data10).commit(); table.newFastAppend().appendFile(data11).commit(); @@ -441,12 +441,6 @@ public void testPartitionSpecEvolutionRemovalV2() { assertThat(tasks).hasSize(expectedScanTaskCount(3)); } - private DeleteFile posDelete(Table table, DataFile dataFile) { - return formatVersion >= 3 - ? FileGenerationUtil.generateDV(table, dataFile) - : FileGenerationUtil.generatePositionDeleteFile(table, dataFile); - } - @TestTemplate public void testPartitionSpecEvolutionAdditiveV1() { assumeThat(formatVersion).isEqualTo(1); @@ -537,8 +531,8 @@ public void testPartitionSpecEvolutionAdditiveV2AndAbove() { .withPartitionPath("data_bucket=1/id=11") .build(); - DeleteFile delete10 = posDelete(table, data10); - DeleteFile delete11 = posDelete(table, data11); + DeleteFile delete10 = newDeletes(data10); + DeleteFile delete11 = newDeletes(data11); table.newFastAppend().appendFile(data10).commit(); table.newFastAppend().appendFile(data11).commit(); diff --git a/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java b/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java index 9d16da124062..ada9e27a2fbe 100644 --- a/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java +++ b/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java @@ -19,6 +19,7 @@ package org.apache.iceberg.data; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -82,12 +83,16 @@ public abstract class DeleteReadTests { @Parameter protected FileFormat format; + @Parameter(index = 1) + protected int formatVersion; + @Parameters(name = "fileFormat = {0}") public static Object[][] parameters() { return new Object[][] { - new Object[] {FileFormat.PARQUET}, - new Object[] {FileFormat.AVRO}, - new Object[] {FileFormat.ORC} + new Object[] {FileFormat.PARQUET, 2}, + new Object[] {FileFormat.AVRO, 2}, + new Object[] {FileFormat.ORC, 2}, + new Object[] {FileFormat.PARQUET, 3}, }; } @@ -384,7 +389,8 @@ public void testPositionDeletes() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -401,6 +407,10 @@ public void testPositionDeletes() throws IOException { @TestTemplate public void testMultiplePosDeleteFiles() throws IOException { + assumeThat(formatVersion) + .as("Can't write multiple delete files with formatVersion >= 3") + .isEqualTo(2); + List> deletes = Lists.newArrayList( Pair.of(dataFile.path(), 0L), // id = 29 @@ -412,7 +422,8 @@ public void testMultiplePosDeleteFiles() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -420,17 +431,15 @@ public void testMultiplePosDeleteFiles() throws IOException { .validateDataFilesExist(posDeletes.second()) .commit(); - deletes = - Lists.newArrayList( - Pair.of(dataFile.path(), 6L) // id = 122 - ); + deletes = Lists.newArrayList(Pair.of(dataFile.path(), 6L)); // id = 122 posDeletes = FileHelpers.writeDeleteFile( table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -475,7 +484,8 @@ public void testMixedPositionAndEqualityDeletes() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericReaderDeletes.java b/data/src/test/java/org/apache/iceberg/data/TestGenericReaderDeletes.java index d7c70919015d..b15f5b70720b 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericReaderDeletes.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericReaderDeletes.java @@ -37,7 +37,7 @@ public class TestGenericReaderDeletes extends DeleteReadTests { @Override protected Table createTable(String name, Schema schema, PartitionSpec spec) throws IOException { - return TestTables.create(tableDir, name, schema, spec, 2); + return TestTables.create(tableDir, name, schema, spec, formatVersion); } @Override diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java index 0b5a8011ad3f..188a44d7cdba 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java @@ -73,7 +73,7 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { Table table = catalog.createTable(TableIdentifier.of(databaseName, name), schema, spec, props); TableOperations ops = ((BaseTable) table).operations(); TableMetadata meta = ops.current(); - ops.commit(meta, meta.upgradeToFormatVersion(2)); + ops.commit(meta, meta.upgradeToFormatVersion(formatVersion)); return table; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java index 0b5a8011ad3f..188a44d7cdba 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java @@ -73,7 +73,7 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { Table table = catalog.createTable(TableIdentifier.of(databaseName, name), schema, spec, props); TableOperations ops = ((BaseTable) table).operations(); TableMetadata meta = ops.current(); - ops.commit(meta, meta.upgradeToFormatVersion(2)); + ops.commit(meta, meta.upgradeToFormatVersion(formatVersion)); return table; } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java index 0b5a8011ad3f..188a44d7cdba 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java @@ -73,7 +73,7 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { Table table = catalog.createTable(TableIdentifier.of(databaseName, name), schema, spec, props); TableOperations ops = ((BaseTable) table).operations(); TableMetadata meta = ops.current(); - ops.commit(meta, meta.upgradeToFormatVersion(2)); + ops.commit(meta, meta.upgradeToFormatVersion(formatVersion)); return table; } diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java b/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java index 2cb41f11295c..ac3efc26d644 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestInputFormatReaderDeletes.java @@ -49,18 +49,20 @@ public class TestInputFormatReaderDeletes extends DeleteReadTests { private final HadoopTables tables = new HadoopTables(conf); private TestHelper helper; - @Parameter(index = 1) + @Parameter(index = 2) private String inputFormat; - @Parameters(name = "fileFormat = {0}, inputFormat = {1}") + @Parameters(name = "fileFormat = {0}, formatVersion = {1}, inputFormat = {2}") public static Object[][] parameters() { return new Object[][] { - {FileFormat.PARQUET, "IcebergInputFormat"}, - {FileFormat.AVRO, "IcebergInputFormat"}, - {FileFormat.ORC, "IcebergInputFormat"}, - {FileFormat.PARQUET, "MapredIcebergInputFormat"}, - {FileFormat.AVRO, "MapredIcebergInputFormat"}, - {FileFormat.ORC, "MapredIcebergInputFormat"}, + {FileFormat.PARQUET, 2, "IcebergInputFormat"}, + {FileFormat.AVRO, 2, "IcebergInputFormat"}, + {FileFormat.ORC, 2, "IcebergInputFormat"}, + {FileFormat.PARQUET, 2, "MapredIcebergInputFormat"}, + {FileFormat.AVRO, 2, "MapredIcebergInputFormat"}, + {FileFormat.ORC, 2, "MapredIcebergInputFormat"}, + {FileFormat.PARQUET, 3, "IcebergInputFormat"}, + {FileFormat.PARQUET, 3, "MapredIcebergInputFormat"}, }; } @@ -82,7 +84,7 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) thro TableOperations ops = ((BaseTable) table).operations(); TableMetadata meta = ops.current(); - ops.commit(meta, meta.upgradeToFormatVersion(2)); + ops.commit(meta, meta.upgradeToFormatVersion(formatVersion)); return table; } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 5ba296e74c35..bbdd49b63df8 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -44,6 +44,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PlanningMode; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -99,16 +100,21 @@ public class TestSparkReaderDeletes extends DeleteReadTests { protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - @Parameter(index = 1) + @Parameter(index = 2) private boolean vectorized; - @Parameters(name = "format = {0}, vectorized = {1}") + @Parameter(index = 3) + private PlanningMode planningMode; + + @Parameters(name = "fileFormat = {0}, formatVersion = {1}, vectorized = {2}, planningMode = {3}") public static Object[][] parameters() { return new Object[][] { - new Object[] {FileFormat.PARQUET, false}, - new Object[] {FileFormat.PARQUET, true}, - new Object[] {FileFormat.ORC, false}, - new Object[] {FileFormat.AVRO, false} + new Object[] {FileFormat.PARQUET, 2, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, 2, true, PlanningMode.LOCAL}, + new Object[] {FileFormat.ORC, 2, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.AVRO, 2, false, PlanningMode.LOCAL}, + new Object[] {FileFormat.PARQUET, 3, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, 3, true, PlanningMode.LOCAL}, }; } @@ -162,7 +168,13 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { TableOperations ops = ((BaseTable) table).operations(); TableMetadata meta = ops.current(); ops.commit(meta, meta.upgradeToFormatVersion(2)); - table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.name()).commit(); + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.DATA_PLANNING_MODE, planningMode.modeName()) + .set(TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()) + .set(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)) + .commit(); if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { String vectorizationEnabled = format.equals(FileFormat.PARQUET) @@ -342,7 +354,8 @@ public void testPosDeletesAllRowsInBatch() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -374,7 +387,8 @@ public void testPosDeletesWithDeletedColumn() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -450,7 +464,8 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -482,7 +497,8 @@ public void testFilterOnDeletedMetadataColumn() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -604,7 +620,10 @@ public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOExceptio Pair.of(dataFile.path(), 109L)); Pair posDeletes = FileHelpers.writeDeleteFile( - table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), deletes); + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + deletes, + formatVersion); tbl.newRowDelta() .addDeletes(posDeletes.first()) .validateDataFilesExist(posDeletes.second()) diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 285a210db5f2..bbdd49b63df8 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -100,19 +100,21 @@ public class TestSparkReaderDeletes extends DeleteReadTests { protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - @Parameter(index = 1) + @Parameter(index = 2) private boolean vectorized; - @Parameter(index = 2) + @Parameter(index = 3) private PlanningMode planningMode; - @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") + @Parameters(name = "fileFormat = {0}, formatVersion = {1}, vectorized = {2}, planningMode = {3}") public static Object[][] parameters() { return new Object[][] { - new Object[] {FileFormat.PARQUET, false, PlanningMode.DISTRIBUTED}, - new Object[] {FileFormat.PARQUET, true, PlanningMode.LOCAL}, - new Object[] {FileFormat.ORC, false, PlanningMode.DISTRIBUTED}, - new Object[] {FileFormat.AVRO, false, PlanningMode.LOCAL} + new Object[] {FileFormat.PARQUET, 2, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, 2, true, PlanningMode.LOCAL}, + new Object[] {FileFormat.ORC, 2, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.AVRO, 2, false, PlanningMode.LOCAL}, + new Object[] {FileFormat.PARQUET, 3, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, 3, true, PlanningMode.LOCAL}, }; } @@ -171,6 +173,7 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) .set(TableProperties.DATA_PLANNING_MODE, planningMode.modeName()) .set(TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()) + .set(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)) .commit(); if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { String vectorizationEnabled = @@ -351,7 +354,8 @@ public void testPosDeletesAllRowsInBatch() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -383,7 +387,8 @@ public void testPosDeletesWithDeletedColumn() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -459,7 +464,8 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -491,7 +497,8 @@ public void testFilterOnDeletedMetadataColumn() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -613,7 +620,10 @@ public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOExceptio Pair.of(dataFile.path(), 109L)); Pair posDeletes = FileHelpers.writeDeleteFile( - table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), deletes); + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + deletes, + formatVersion); tbl.newRowDelta() .addDeletes(posDeletes.first()) .validateDataFilesExist(posDeletes.second()) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 285a210db5f2..d49b450524d9 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -100,19 +100,21 @@ public class TestSparkReaderDeletes extends DeleteReadTests { protected static SparkSession spark = null; protected static HiveCatalog catalog = null; - @Parameter(index = 1) + @Parameter(index = 2) private boolean vectorized; - @Parameter(index = 2) + @Parameter(index = 3) private PlanningMode planningMode; - @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") + @Parameters(name = "fileFormat = {0}, formatVersion = {1}, vectorized = {2}, planningMode = {3}") public static Object[][] parameters() { return new Object[][] { - new Object[] {FileFormat.PARQUET, false, PlanningMode.DISTRIBUTED}, - new Object[] {FileFormat.PARQUET, true, PlanningMode.LOCAL}, - new Object[] {FileFormat.ORC, false, PlanningMode.DISTRIBUTED}, - new Object[] {FileFormat.AVRO, false, PlanningMode.LOCAL} + new Object[] {FileFormat.PARQUET, 2, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, 2, true, PlanningMode.LOCAL}, + new Object[] {FileFormat.ORC, 2, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.AVRO, 2, false, PlanningMode.LOCAL}, + new Object[] {FileFormat.PARQUET, 3, false, PlanningMode.DISTRIBUTED}, + new Object[] {FileFormat.PARQUET, 3, true, PlanningMode.LOCAL}, }; } @@ -165,12 +167,13 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) { Table table = catalog.createTable(TableIdentifier.of("default", name), schema); TableOperations ops = ((BaseTable) table).operations(); TableMetadata meta = ops.current(); - ops.commit(meta, meta.upgradeToFormatVersion(2)); + ops.commit(meta, meta.upgradeToFormatVersion(formatVersion)); table .updateProperties() .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) .set(TableProperties.DATA_PLANNING_MODE, planningMode.modeName()) .set(TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()) + .set(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)) .commit(); if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { String vectorizationEnabled = @@ -351,7 +354,8 @@ public void testPosDeletesAllRowsInBatch() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -383,7 +387,8 @@ public void testPosDeletesWithDeletedColumn() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -459,7 +464,8 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -491,7 +497,8 @@ public void testFilterOnDeletedMetadataColumn() throws IOException { table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), TestHelpers.Row.of(0), - deletes); + deletes, + formatVersion); table .newRowDelta() @@ -613,7 +620,10 @@ public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOExceptio Pair.of(dataFile.path(), 109L)); Pair posDeletes = FileHelpers.writeDeleteFile( - table, Files.localOutput(File.createTempFile("junit", null, temp.toFile())), deletes); + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + deletes, + formatVersion); tbl.newRowDelta() .addDeletes(posDeletes.first()) .validateDataFilesExist(posDeletes.second()) From 491718c3e22eff592d6924c70496f5b4eeb7a978 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Nov 2024 11:33:42 +0100 Subject: [PATCH 153/313] Build: Bump nessie from 0.99.0 to 0.100.0 (#11567) Bumps `nessie` from 0.99.0 to 0.100.0. Updates `org.projectnessie.nessie:nessie-client` from 0.99.0 to 0.100.0 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.99.0 to 0.100.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.99.0 to 0.100.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.99.0 to 0.100.0 --- updated-dependencies: - dependency-name: org.projectnessie.nessie:nessie-client dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-jaxrs-testextension dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-testextension dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 96f62fb8163b..8008a886a8af 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -70,7 +70,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.99.0" +nessie = "0.100.0" netty-buffer = "4.1.115.Final" netty-buffer-compat = "4.1.115.Final" object-client-bundle = "3.3.2" From f9256c698107b0303eb3d33c3f7fa8c3ed87b020 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Nov 2024 11:37:05 +0100 Subject: [PATCH 154/313] Build: Bump orc from 1.9.4 to 1.9.5 (#11571) Bumps `orc` from 1.9.4 to 1.9.5. Updates `org.apache.orc:orc-core` from 1.9.4 to 1.9.5 Updates `org.apache.orc:orc-tools` from 1.9.4 to 1.9.5 --- updated-dependencies: - dependency-name: org.apache.orc:orc-core dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.orc:orc-tools dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 8008a886a8af..8af0d6ec6ab2 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -74,7 +74,7 @@ nessie = "0.100.0" netty-buffer = "4.1.115.Final" netty-buffer-compat = "4.1.115.Final" object-client-bundle = "3.3.2" -orc = "1.9.4" +orc = "1.9.5" parquet = "1.13.1" roaringbitmap = "1.3.0" scala-collection-compat = "2.12.0" From 97542ab677cc1cc03f9cab7461266f01888ef1f9 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 18 Nov 2024 08:26:58 -0700 Subject: [PATCH 155/313] API, Arrow, Core, Data, Spark: Replace usage of deprecated ContentFile#path API with location API (#11563) --- .../java/org/apache/iceberg/TestHelpers.java | 2 +- .../iceberg/arrow/vectorized/ArrowReader.java | 4 +- .../encryption/InputFilesDecryptor.java | 2 +- .../ScanPlanningAndReportingTestBase.java | 2 +- .../java/org/apache/iceberg/TestSnapshot.java | 8 +- .../org/apache/iceberg/TestTransaction.java | 9 +-- .../apache/iceberg/catalog/CatalogTests.java | 5 +- .../apache/iceberg/data/BaseDeleteLoader.java | 6 +- .../org/apache/iceberg/data/DeleteFilter.java | 6 +- .../iceberg/data/GenericDeleteFilter.java | 2 +- .../apache/iceberg/data/GenericReader.java | 4 +- .../apache/iceberg/data/DeleteReadTests.java | 16 ++-- .../data/TestDataFileIndexStatsFilters.java | 16 ++-- .../iceberg/io/TestAppenderFactory.java | 24 +++--- .../iceberg/io/TestFileWriterFactory.java | 30 ++++---- .../io/TestGenericSortedPosDeleteWriter.java | 29 ++++---- .../iceberg/io/TestPartitioningWriters.java | 58 ++++++++------- .../iceberg/io/TestPositionDeltaWriters.java | 10 ++- .../delta/TestSnapshotDeltaLakeTable.java | 4 +- .../BaseSnapshotDeltaLakeTableAction.java | 4 +- .../TestExpireSnapshotsProcedure.java | 2 +- .../TestRemoveOrphanFilesProcedure.java | 2 +- .../TestRewritePositionDeleteFiles.java | 2 +- .../iceberg/spark/extensions/TestUpdate.java | 2 +- .../spark/actions/BaseSparkAction.java | 2 +- .../RemoveDanglingDeletesSparkAction.java | 2 +- .../iceberg/spark/source/BaseReader.java | 2 +- .../iceberg/spark/source/BatchDataReader.java | 2 +- .../spark/source/ChangelogRowReader.java | 2 +- .../spark/source/EqualityDeleteRowReader.java | 4 +- .../source/PositionDeletesRowReader.java | 4 +- .../iceberg/spark/source/RowDataReader.java | 4 +- .../spark/source/SparkCleanupUtil.java | 2 +- .../spark/source/SparkCopyOnWriteScan.java | 2 +- .../org/apache/iceberg/TaskCheckHelper.java | 6 +- .../org/apache/iceberg/ValidationHelpers.java | 4 +- .../org/apache/iceberg/spark/TestBase.java | 2 +- .../iceberg/spark/TestSparkExecutorCache.java | 2 +- .../TestDeleteReachableFilesAction.java | 2 +- .../actions/TestExpireSnapshotsAction.java | 32 ++++---- .../iceberg/spark/source/TestBaseReader.java | 2 +- .../spark/source/TestCompressionSettings.java | 6 +- .../spark/source/TestDataFrameWrites.java | 4 +- .../spark/source/TestDataSourceOptions.java | 4 +- .../source/TestPositionDeletesTable.java | 73 ++++++++++--------- .../spark/source/TestRuntimeFiltering.java | 4 +- .../spark/source/TestSparkDataFile.java | 6 +- .../spark/source/TestSparkReaderDeletes.java | 42 +++++------ 48 files changed, 237 insertions(+), 227 deletions(-) diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index 003e7835ed4b..cf1dc203f4d6 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -649,7 +649,7 @@ public CharSequence path() { @Override public FileFormat format() { - return FileFormat.fromFileName(path()); + return FileFormat.fromFileName(location()); } @Override diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java index dd2b1e0221e4..7bd744ec5bf3 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java @@ -260,7 +260,7 @@ private static final class VectorizedCombinedScanIterator Map keyMetadata = Maps.newHashMap(); fileTasks.stream() .map(FileScanTask::file) - .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata())); + .forEach(file -> keyMetadata.put(file.location(), file.keyMetadata())); Stream encrypted = keyMetadata.entrySet().stream() @@ -364,7 +364,7 @@ public void close() throws IOException { private InputFile getInputFile(FileScanTask task) { Preconditions.checkArgument(!task.isDataTask(), "Invalid task type"); - return inputFiles.get(task.file().path().toString()); + return inputFiles.get(task.file().location()); } /** diff --git a/core/src/main/java/org/apache/iceberg/encryption/InputFilesDecryptor.java b/core/src/main/java/org/apache/iceberg/encryption/InputFilesDecryptor.java index 44cd38c58621..678d405fd7c9 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/InputFilesDecryptor.java +++ b/core/src/main/java/org/apache/iceberg/encryption/InputFilesDecryptor.java @@ -40,7 +40,7 @@ public InputFilesDecryptor( .flatMap( fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream())) - .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata())); + .forEach(file -> keyMetadata.put(file.location(), file.keyMetadata())); Stream encrypted = keyMetadata.entrySet().stream() .map( diff --git a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java index f7c287f1dc04..7f7d33a7b54d 100644 --- a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java @@ -282,7 +282,7 @@ public void scanningWithSkippedDeleteFiles() throws IOException { } assertThat(fileTasks) .singleElement() - .satisfies(task -> assertThat(task.file().path()).isEqualTo(FILE_D.path())); + .satisfies(task -> assertThat(task.file().location()).isEqualTo(FILE_D.location())); ScanReport scanReport = reporter.lastReport(); assertThat(scanReport).isNotNull(); diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java index bbe5e8f6cdd8..59217e5b91c8 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java @@ -99,7 +99,7 @@ public void testCachedDataFiles() { assertThat(removedDataFiles).as("Must have 1 removed data file").hasSize(1); DataFile removedDataFile = Iterables.getOnlyElement(removedDataFiles); - assertThat(removedDataFile.path()).isEqualTo(FILE_A.path()); + assertThat(removedDataFile.location()).isEqualTo(FILE_A.location()); assertThat(removedDataFile.specId()).isEqualTo(FILE_A.specId()); assertThat(removedDataFile.partition()).isEqualTo(FILE_A.partition()); @@ -107,7 +107,7 @@ public void testCachedDataFiles() { assertThat(addedDataFiles).as("Must have 1 added data file").hasSize(1); DataFile addedDataFile = Iterables.getOnlyElement(addedDataFiles); - assertThat(addedDataFile.path()).isEqualTo(thirdSnapshotDataFile.path()); + assertThat(addedDataFile.location()).isEqualTo(thirdSnapshotDataFile.location()); assertThat(addedDataFile.specId()).isEqualTo(thirdSnapshotDataFile.specId()); assertThat(addedDataFile.partition()).isEqualTo(thirdSnapshotDataFile.partition()); } @@ -147,7 +147,7 @@ public void testCachedDeleteFiles() { assertThat(removedDeleteFiles).as("Must have 1 removed delete file").hasSize(1); DeleteFile removedDeleteFile = Iterables.getOnlyElement(removedDeleteFiles); - assertThat(removedDeleteFile.path()).isEqualTo(secondSnapshotDeleteFile.path()); + assertThat(removedDeleteFile.location()).isEqualTo(secondSnapshotDeleteFile.location()); assertThat(removedDeleteFile.specId()).isEqualTo(secondSnapshotDeleteFile.specId()); assertThat(removedDeleteFile.partition()).isEqualTo(secondSnapshotDeleteFile.partition()); @@ -155,7 +155,7 @@ public void testCachedDeleteFiles() { assertThat(addedDeleteFiles).as("Must have 1 added delete file").hasSize(1); DeleteFile addedDeleteFile = Iterables.getOnlyElement(addedDeleteFiles); - assertThat(addedDeleteFile.path()).isEqualTo(thirdSnapshotDeleteFile.path()); + assertThat(addedDeleteFile.location()).isEqualTo(thirdSnapshotDeleteFile.location()); assertThat(addedDeleteFile.specId()).isEqualTo(thirdSnapshotDeleteFile.specId()); assertThat(addedDeleteFile.partition()).isEqualTo(thirdSnapshotDeleteFile.partition()); } diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 8770e24f8e40..2042906c15c6 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -702,14 +702,9 @@ public void testTransactionRecommit() { Set paths = Sets.newHashSet( - Iterables.transform( - table.newScan().planFiles(), task -> task.file().path().toString())); + Iterables.transform(table.newScan().planFiles(), task -> task.file().location())); Set expectedPaths = - Sets.newHashSet( - FILE_A.path().toString(), - FILE_B.path().toString(), - FILE_C.path().toString(), - FILE_D.path().toString()); + Sets.newHashSet(FILE_A.location(), FILE_B.location(), FILE_C.location(), FILE_D.location()); assertThat(paths).isEqualTo(expectedPaths); assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); diff --git a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java index 75983402a38c..a011578865b4 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -2729,12 +2729,13 @@ public void assertFiles(Table table, DataFile... files) { List paths = Streams.stream(tasks) .map(FileScanTask::file) - .map(DataFile::path) + .map(DataFile::location) .collect(Collectors.toList()); assertThat(paths).as("Should contain expected number of data files").hasSize(files.length); assertThat(CharSequenceSet.of(paths)) .as("Should contain correct file paths") - .isEqualTo(CharSequenceSet.of(Iterables.transform(Arrays.asList(files), DataFile::path))); + .isEqualTo( + CharSequenceSet.of(Iterables.transform(Arrays.asList(files), DataFile::location))); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java index 796f4f92be33..1b7a92f0682b 100644 --- a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java +++ b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java @@ -119,7 +119,7 @@ public StructLikeSet loadEqualityDeletes(Iterable deleteFiles, Schem private Iterable getOrReadEqDeletes(DeleteFile deleteFile, Schema projection) { long estimatedSize = estimateEqDeletesSize(deleteFile, projection); if (canCache(estimatedSize)) { - String cacheKey = deleteFile.path().toString(); + String cacheKey = deleteFile.location(); return getOrLoad(cacheKey, () -> readEqDeletes(deleteFile, projection), estimatedSize); } else { return readEqDeletes(deleteFile, projection); @@ -199,7 +199,7 @@ private PositionDeleteIndex getOrReadPosDeletes( private PositionDeleteIndex getOrReadPosDeletes(DeleteFile deleteFile, CharSequence filePath) { long estimatedSize = estimatePosDeletesSize(deleteFile); if (canCache(estimatedSize)) { - String cacheKey = deleteFile.path().toString(); + String cacheKey = deleteFile.location(); CharSequenceMap indexes = getOrLoad(cacheKey, () -> readPosDeletes(deleteFile), estimatedSize); return indexes.getOrDefault(filePath, PositionDeleteIndex.empty()); @@ -227,7 +227,7 @@ private CloseableIterable openDeletes( DeleteFile deleteFile, Schema projection, Expression filter) { FileFormat format = deleteFile.format(); - LOG.trace("Opening delete file {}", deleteFile.path()); + LOG.trace("Opening delete file {}", deleteFile.location()); InputFile inputFile = loadInputFile.apply(deleteFile); switch (format) { diff --git a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java index aa5e00fd0ef4..96ee6fd3f5ba 100644 --- a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java +++ b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java @@ -79,11 +79,11 @@ protected DeleteFilter( for (DeleteFile delete : deletes) { switch (delete.content()) { case POSITION_DELETES: - LOG.debug("Adding position delete file {} to filter", delete.path()); + LOG.debug("Adding position delete file {} to filter", delete.location()); posDeleteBuilder.add(delete); break; case EQUALITY_DELETES: - LOG.debug("Adding equality delete file {} to filter", delete.path()); + LOG.debug("Adding equality delete file {} to filter", delete.location()); eqDeleteBuilder.add(delete); break; default: @@ -145,7 +145,7 @@ Accessor posAccessor() { protected abstract InputFile getInputFile(String location); protected InputFile loadInputFile(DeleteFile deleteFile) { - return getInputFile(deleteFile.path().toString()); + return getInputFile(deleteFile.location()); } protected long pos(T record) { diff --git a/data/src/main/java/org/apache/iceberg/data/GenericDeleteFilter.java b/data/src/main/java/org/apache/iceberg/data/GenericDeleteFilter.java index 0779ed09ce1e..712263e40bc2 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericDeleteFilter.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeleteFilter.java @@ -30,7 +30,7 @@ public class GenericDeleteFilter extends DeleteFilter { public GenericDeleteFilter( FileIO io, FileScanTask task, Schema tableSchema, Schema requestedSchema) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + super(task.file().location(), task.deletes(), tableSchema, requestedSchema); this.io = io; this.asStructLike = new InternalRecordWrapper(requiredSchema().asStruct()); } diff --git a/data/src/main/java/org/apache/iceberg/data/GenericReader.java b/data/src/main/java/org/apache/iceberg/data/GenericReader.java index bf2919f334a8..590b01b228ed 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericReader.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericReader.java @@ -92,7 +92,7 @@ private CloseableIterable applyResidual( } private CloseableIterable openFile(FileScanTask task, Schema fileProjection) { - InputFile input = io.newInputFile(task.file().path().toString()); + InputFile input = io.newInputFile(task.file().location()); Map partition = PartitionUtil.constantsMap(task, IdentityPartitionConverters::convertConstant); @@ -147,7 +147,7 @@ private CloseableIterable openFile(FileScanTask task, Schema fileProject default: throw new UnsupportedOperationException( String.format( - "Cannot read %s file: %s", task.file().format().name(), task.file().path())); + "Cannot read %s file: %s", task.file().format().name(), task.file().location())); } } diff --git a/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java b/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java index ada9e27a2fbe..7c2621494288 100644 --- a/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java +++ b/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java @@ -379,9 +379,9 @@ public void testEqualityDeletesSpanningMultipleDataFiles() throws IOException { public void testPositionDeletes() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 3L), // id = 89 - Pair.of(dataFile.path(), 6L) // id = 122 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 3L), // id = 89 + Pair.of(dataFile.location(), 6L) // id = 122 ); Pair posDeletes = @@ -413,8 +413,8 @@ public void testMultiplePosDeleteFiles() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -431,7 +431,7 @@ public void testMultiplePosDeleteFiles() throws IOException { .validateDataFilesExist(posDeletes.second()) .commit(); - deletes = Lists.newArrayList(Pair.of(dataFile.path(), 6L)); // id = 122 + deletes = Lists.newArrayList(Pair.of(dataFile.location(), 6L)); // id = 122 posDeletes = FileHelpers.writeDeleteFile( @@ -475,8 +475,8 @@ public void testMixedPositionAndEqualityDeletes() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 3L), // id = 89 - Pair.of(dataFile.path(), 5L) // id = 121 + Pair.of(dataFile.location(), 3L), // id = 89 + Pair.of(dataFile.location(), 5L) // id = 121 ); Pair posDeletes = diff --git a/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java b/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java index 0c97d9ecd4da..2cbf2ad4d1f0 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java +++ b/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java @@ -116,12 +116,12 @@ public void dropTable() { } @Test - public void testPositionDeletePlanningPath() throws IOException { + public void testPositionDeletePlanninglocation() throws IOException { table.newAppend().appendFile(dataFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dataFile.path(), 0L)); - deletes.add(Pair.of(dataFile.path(), 1L)); + deletes.add(Pair.of(dataFile.location(), 0L)); + deletes.add(Pair.of(dataFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile(table, Files.localOutput(createTempFile()), deletes); @@ -383,7 +383,7 @@ public void testDifferentDeleteTypes() throws IOException { writePosDeletes( evenPartition, ImmutableList.of( - Pair.of(dataFileWithEvenRecords.path(), 0L), + Pair.of(dataFileWithEvenRecords.location(), 0L), Pair.of("some-other-file.parquet", 0L))); table .newRowDelta() @@ -396,8 +396,8 @@ public void testDifferentDeleteTypes() throws IOException { writePosDeletes( evenPartition, ImmutableList.of( - Pair.of(dataFileWithEvenRecords.path(), 1L), - Pair.of(dataFileWithEvenRecords.path(), 2L))); + Pair.of(dataFileWithEvenRecords.location(), 1L), + Pair.of(dataFileWithEvenRecords.location(), 2L))); table .newRowDelta() .addDeletes(pathPosDeletes.first()) @@ -437,7 +437,7 @@ public void testDifferentDeleteTypes() throws IOException { } private boolean coversDataFile(FileScanTask task, DataFile file) { - return task.file().path().toString().equals(file.path().toString()); + return task.file().location().toString().equals(file.location().toString()); } private void assertDeletes(FileScanTask task, DeleteFile... expectedDeleteFiles) { @@ -446,7 +446,7 @@ private void assertDeletes(FileScanTask task, DeleteFile... expectedDeleteFiles) assertThat(actualDeletePaths.size()).isEqualTo(expectedDeleteFiles.length); for (DeleteFile expectedDeleteFile : expectedDeleteFiles) { - assertThat(actualDeletePaths.contains(expectedDeleteFile.path())).isTrue(); + assertThat(actualDeletePaths.contains(expectedDeleteFile.location())).isTrue(); } } diff --git a/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java b/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java index f3f25ff317f6..83f1bf261063 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java @@ -219,9 +219,9 @@ public void testPosDeleteWriter() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), - Pair.of(dataFile.path(), 2L), - Pair.of(dataFile.path(), 4L)); + Pair.of(dataFile.location(), 0L), + Pair.of(dataFile.location(), 2L), + Pair.of(dataFile.location(), 4L)); EncryptedOutputFile out = createEncryptedOutputFile(); PositionDeleteWriter eqDeleteWriter = @@ -238,9 +238,9 @@ public void testPosDeleteWriter() throws IOException { GenericRecord gRecord = GenericRecord.create(pathPosSchema); Set expectedDeletes = Sets.newHashSet( - gRecord.copy("file_path", dataFile.path(), "pos", 0L), - gRecord.copy("file_path", dataFile.path(), "pos", 2L), - gRecord.copy("file_path", dataFile.path(), "pos", 4L)); + gRecord.copy("file_path", dataFile.location(), "pos", 0L), + gRecord.copy("file_path", dataFile.location(), "pos", 2L), + gRecord.copy("file_path", dataFile.location(), "pos", 4L)); assertThat( Sets.newHashSet(createReader(pathPosSchema, out.encryptingOutputFile().toInputFile()))) .isEqualTo(expectedDeletes); @@ -268,9 +268,9 @@ public void testPosDeleteWriterWithRowSchema() throws IOException { List> deletes = Lists.newArrayList( - positionDelete(dataFile.path(), 0, rowSet.get(0)), - positionDelete(dataFile.path(), 2, rowSet.get(2)), - positionDelete(dataFile.path(), 4, rowSet.get(4))); + positionDelete(dataFile.location(), 0, rowSet.get(0)), + positionDelete(dataFile.location(), 2, rowSet.get(2)), + positionDelete(dataFile.location(), 4, rowSet.get(4))); EncryptedOutputFile out = createEncryptedOutputFile(); PositionDeleteWriter eqDeleteWriter = @@ -290,21 +290,21 @@ public void testPosDeleteWriterWithRowSchema() throws IOException { Sets.newHashSet( gRecord.copy( "file_path", - dataFile.path(), + dataFile.location(), "pos", 0L, "row", rowRecord.copy("id", 1, "data", "aaa")), gRecord.copy( "file_path", - dataFile.path(), + dataFile.location(), "pos", 2L, "row", rowRecord.copy("id", 3, "data", "ccc")), gRecord.copy( "file_path", - dataFile.path(), + dataFile.location(), "pos", 4L, "row", diff --git a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java index a0980b650b3b..0acb173f0923 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java @@ -144,7 +144,7 @@ public void testEqualityDeleteWriter() throws IOException { List expectedDeletes = ImmutableList.of( deleteRecord.copy("id", 1), deleteRecord.copy("id", 3), deleteRecord.copy("id", 5)); - InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); + InputFile inputDeleteFile = table.io().newInputFile(deleteFile.location()); List actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile); assertThat(actualDeletes).isEqualTo(expectedDeletes); @@ -222,9 +222,9 @@ public void testPositionDeleteWriter() throws IOException { // write a position delete file List> deletes = ImmutableList.of( - positionDelete(dataFile.path(), 0L, null), - positionDelete(dataFile.path(), 2L, null), - positionDelete(dataFile.path(), 4L, null)); + positionDelete(dataFile.location(), 0L, null), + positionDelete(dataFile.location(), 2L, null), + positionDelete(dataFile.location(), 4L, null)); Pair result = writePositionDeletes(writerFactory, deletes, table.spec(), partition); DeleteFile deleteFile = result.first(); @@ -249,11 +249,13 @@ public void testPositionDeleteWriter() throws IOException { GenericRecord deleteRecord = GenericRecord.create(DeleteSchemaUtil.pathPosSchema()); List expectedDeletes = ImmutableList.of( - deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 0L), - deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 2L), deleteRecord.copy( - DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 4L)); - InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); + DELETE_FILE_PATH.name(), dataFile.location(), DELETE_FILE_POS.name(), 0L), + deleteRecord.copy( + DELETE_FILE_PATH.name(), dataFile.location(), DELETE_FILE_POS.name(), 2L), + deleteRecord.copy( + DELETE_FILE_PATH.name(), dataFile.location(), DELETE_FILE_POS.name(), 4L)); + InputFile inputDeleteFile = table.io().newInputFile(deleteFile.location()); List actualDeletes = readFile(DeleteSchemaUtil.pathPosSchema(), inputDeleteFile); assertThat(actualDeletes).isEqualTo(expectedDeletes); @@ -280,7 +282,7 @@ public void testPositionDeleteWriterWithRow() throws IOException { // write a position delete file and persist the deleted row List> deletes = - ImmutableList.of(positionDelete(dataFile.path(), 0, dataRows.get(0))); + ImmutableList.of(positionDelete(dataFile.location(), 0, dataRows.get(0))); Pair result = writePositionDeletes(writerFactory, deletes, table.spec(), partition); DeleteFile deleteFile = result.first(); @@ -323,13 +325,13 @@ public void testPositionDeleteWriterWithRow() throws IOException { Map deleteRecordColumns = ImmutableMap.of( DELETE_FILE_PATH.name(), - dataFile.path(), + dataFile.location(), DELETE_FILE_POS.name(), 0L, DELETE_FILE_ROW_FIELD_NAME, deletedRow.copy("id", 1, "data", "aaa")); List expectedDeletes = ImmutableList.of(deleteRecord.copy(deleteRecordColumns)); - InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); + InputFile inputDeleteFile = table.io().newInputFile(deleteFile.location()); List actualDeletes = readFile(positionDeleteSchema, inputDeleteFile); assertThat(actualDeletes).isEqualTo(expectedDeletes); @@ -359,9 +361,9 @@ public void testPositionDeleteWriterMultipleDataFiles() throws IOException { // write a position delete file referencing both List> deletes = ImmutableList.of( - positionDelete(dataFile1.path(), 0L, null), - positionDelete(dataFile1.path(), 2L, null), - positionDelete(dataFile2.path(), 4L, null)); + positionDelete(dataFile1.location(), 0L, null), + positionDelete(dataFile1.location(), 2L, null), + positionDelete(dataFile2.location(), 4L, null)); Pair result = writePositionDeletes(writerFactory, deletes, table.spec(), partition); DeleteFile deleteFile = result.first(); diff --git a/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java b/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java index 5718eb37030d..629df03e37cd 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java @@ -137,7 +137,7 @@ public void testSortedPosDelete() throws IOException { new SortedPosDeleteWriter<>(appenderFactory, fileFactory, format, null, 100); try (SortedPosDeleteWriter closeableWriter = writer) { for (int index = rowSet.size() - 1; index >= 0; index -= 2) { - closeableWriter.delete(dataFile.path(), index); + closeableWriter.delete(dataFile.location(), index); } } @@ -150,10 +150,10 @@ public void testSortedPosDelete() throws IOException { Record record = GenericRecord.create(pathPosSchema); List expectedDeletes = Lists.newArrayList( - record.copy("file_path", dataFile.path(), "pos", 0L), - record.copy("file_path", dataFile.path(), "pos", 2L), - record.copy("file_path", dataFile.path(), "pos", 4L)); - assertThat(readRecordsAsList(pathPosSchema, deleteFile.path())).isEqualTo(expectedDeletes); + record.copy("file_path", dataFile.location(), "pos", 0L), + record.copy("file_path", dataFile.location(), "pos", 2L), + record.copy("file_path", dataFile.location(), "pos", 4L)); + assertThat(readRecordsAsList(pathPosSchema, deleteFile.location())).isEqualTo(expectedDeletes); table .newRowDelta() @@ -181,7 +181,7 @@ public void testSortedPosDeleteWithSchemaAndNullRow() throws IOException { assertThatThrownBy( () -> new SortedPosDeleteWriter<>(appenderFactory, fileFactory, format, null, 1) - .delete(dataFile.path(), 0L)) + .delete(dataFile.location(), 0L)) .isInstanceOf(Exception.class); } @@ -204,7 +204,7 @@ public void testSortedPosDeleteWithRow() throws IOException { try (SortedPosDeleteWriter closeableWriter = writer) { for (int index = rowSet.size() - 1; index >= 0; index -= 2) { closeableWriter.delete( - dataFile.path(), index, rowSet.get(index)); // Write deletes with row. + dataFile.location(), index, rowSet.get(index)); // Write deletes with row. } } @@ -217,10 +217,10 @@ public void testSortedPosDeleteWithRow() throws IOException { Record record = GenericRecord.create(pathPosSchema); List expectedDeletes = Lists.newArrayList( - record.copy("file_path", dataFile.path(), "pos", 0L, "row", createRow(0, "aaa")), - record.copy("file_path", dataFile.path(), "pos", 2L, "row", createRow(2, "ccc")), - record.copy("file_path", dataFile.path(), "pos", 4L, "row", createRow(4, "eee"))); - assertThat(readRecordsAsList(pathPosSchema, deleteFile.path())).isEqualTo(expectedDeletes); + record.copy("file_path", dataFile.location(), "pos", 0L, "row", createRow(0, "aaa")), + record.copy("file_path", dataFile.location(), "pos", 2L, "row", createRow(2, "ccc")), + record.copy("file_path", dataFile.location(), "pos", 4L, "row", createRow(4, "eee"))); + assertThat(readRecordsAsList(pathPosSchema, deleteFile.location())).isEqualTo(expectedDeletes); table .newRowDelta() @@ -267,7 +267,7 @@ public void testMultipleFlush() throws IOException { try (SortedPosDeleteWriter closeableWriter = writer) { for (int pos = 0; pos < 100; pos++) { for (int fileIndex = 4; fileIndex >= 0; fileIndex--) { - closeableWriter.delete(dataFiles.get(fileIndex).path(), pos); + closeableWriter.delete(dataFiles.get(fileIndex).location(), pos); } } } @@ -282,12 +282,13 @@ public void testMultipleFlush() throws IOException { for (int dataFileIndex = 0; dataFileIndex < 5; dataFileIndex++) { DataFile dataFile = dataFiles.get(dataFileIndex); for (long pos = deleteFileIndex * 10; pos < deleteFileIndex * 10 + 10; pos++) { - expectedDeletes.add(record.copy("file_path", dataFile.path(), "pos", pos)); + expectedDeletes.add(record.copy("file_path", dataFile.location(), "pos", pos)); } } DeleteFile deleteFile = deleteFiles.get(deleteFileIndex); - assertThat(readRecordsAsList(pathPosSchema, deleteFile.path())).isEqualTo(expectedDeletes); + assertThat(readRecordsAsList(pathPosSchema, deleteFile.location())) + .isEqualTo(expectedDeletes); } rowDelta = table.newRowDelta(); diff --git a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java index 9813fb0caba2..e404b7008834 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java @@ -362,14 +362,18 @@ private void checkClusteredPositionDeleteWriterMultipleSpecs(DeleteGranularity d PartitionSpec bucketSpec = table.specs().get(1); PartitionSpec identitySpec = table.specs().get(2); - writer.write(positionDelete(dataFile1.path(), 0L, null), unpartitionedSpec, null); - writer.write(positionDelete(dataFile1.path(), 1L, null), unpartitionedSpec, null); + writer.write(positionDelete(dataFile1.location(), 0L, null), unpartitionedSpec, null); + writer.write(positionDelete(dataFile1.location(), 1L, null), unpartitionedSpec, null); writer.write( - positionDelete(dataFile2.path(), 0L, null), bucketSpec, partitionKey(bucketSpec, "bbb")); + positionDelete(dataFile2.location(), 0L, null), + bucketSpec, + partitionKey(bucketSpec, "bbb")); writer.write( - positionDelete(dataFile2.path(), 1L, null), bucketSpec, partitionKey(bucketSpec, "bbb")); + positionDelete(dataFile2.location(), 1L, null), + bucketSpec, + partitionKey(bucketSpec, "bbb")); writer.write( - positionDelete(dataFile3.path(), 0L, null), + positionDelete(dataFile3.location(), 0L, null), identitySpec, partitionKey(identitySpec, "ccc")); @@ -488,10 +492,10 @@ private void checkClusteredPositionDeleteWriterGranularity(DeleteGranularity del PartitionSpec spec = table.spec(); // write deletes for both data files - writer.write(positionDelete(dataFile1.path(), 0L, null), spec, null); - writer.write(positionDelete(dataFile1.path(), 1L, null), spec, null); - writer.write(positionDelete(dataFile2.path(), 0L, null), spec, null); - writer.write(positionDelete(dataFile2.path(), 1L, null), spec, null); + writer.write(positionDelete(dataFile1.location(), 0L, null), spec, null); + writer.write(positionDelete(dataFile1.location(), 1L, null), spec, null); + writer.write(positionDelete(dataFile2.location(), 0L, null), spec, null); + writer.write(positionDelete(dataFile2.location(), 1L, null), spec, null); writer.close(); // verify the writer result @@ -636,25 +640,29 @@ private void checkFanoutPositionOnlyDeleteWriterOutOfOrderRecords( PartitionSpec bucketSpec = table.specs().get(1); PartitionSpec identitySpec = table.specs().get(2); - writer.write(positionDelete(dataFile1.path(), 1L, null), unpartitionedSpec, null); + writer.write(positionDelete(dataFile1.location(), 1L, null), unpartitionedSpec, null); writer.write( - positionDelete(dataFile2.path(), 1L, null), bucketSpec, partitionKey(bucketSpec, "bbb")); + positionDelete(dataFile2.location(), 1L, null), + bucketSpec, + partitionKey(bucketSpec, "bbb")); writer.write( - positionDelete(dataFile2.path(), 0L, null), bucketSpec, partitionKey(bucketSpec, "bbb")); + positionDelete(dataFile2.location(), 0L, null), + bucketSpec, + partitionKey(bucketSpec, "bbb")); writer.write( - positionDelete(dataFile3.path(), 1L, null), + positionDelete(dataFile3.location(), 1L, null), identitySpec, partitionKey(identitySpec, "ccc")); writer.write( - positionDelete(dataFile3.path(), 2L, null), + positionDelete(dataFile3.location(), 2L, null), identitySpec, partitionKey(identitySpec, "ccc")); - writer.write(positionDelete(dataFile1.path(), 0L, null), unpartitionedSpec, null); + writer.write(positionDelete(dataFile1.location(), 0L, null), unpartitionedSpec, null); writer.write( - positionDelete(dataFile3.path(), 0L, null), + positionDelete(dataFile3.location(), 0L, null), identitySpec, partitionKey(identitySpec, "ccc")); - writer.write(positionDelete(dataFile1.path(), 2L, null), unpartitionedSpec, null); + writer.write(positionDelete(dataFile1.location(), 2L, null), unpartitionedSpec, null); writer.close(); @@ -703,10 +711,10 @@ private void checkFanoutPositionOnlyDeleteWriterGranularity(DeleteGranularity de PartitionSpec spec = table.spec(); // write deletes for both data files (the order of records is mixed) - writer.write(positionDelete(dataFile1.path(), 1L, null), spec, null); - writer.write(positionDelete(dataFile2.path(), 0L, null), spec, null); - writer.write(positionDelete(dataFile1.path(), 0L, null), spec, null); - writer.write(positionDelete(dataFile2.path(), 1L, null), spec, null); + writer.write(positionDelete(dataFile1.location(), 1L, null), spec, null); + writer.write(positionDelete(dataFile2.location(), 0L, null), spec, null); + writer.write(positionDelete(dataFile1.location(), 0L, null), spec, null); + writer.write(positionDelete(dataFile2.location(), 1L, null), spec, null); writer.close(); // verify the writer result @@ -750,8 +758,8 @@ public void testRewriteOfPreviousDeletes() throws IOException { writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE, DeleteGranularity.FILE); // write initial deletes for both data files - writer1.write(positionDelete(dataFile1.path(), 1L), spec, null); - writer1.write(positionDelete(dataFile2.path(), 1L), spec, null); + writer1.write(positionDelete(dataFile1.location(), 1L), spec, null); + writer1.write(positionDelete(dataFile2.location(), 1L), spec, null); writer1.close(); // verify the writer result @@ -789,8 +797,8 @@ public void testRewriteOfPreviousDeletes() throws IOException { new PreviousDeleteLoader(table, previousDeletes)); // write more deletes for both data files - writer2.write(positionDelete(dataFile1.path(), 0L), spec, null); - writer2.write(positionDelete(dataFile2.path(), 0L), spec, null); + writer2.write(positionDelete(dataFile1.location(), 0L), spec, null); + writer2.write(positionDelete(dataFile2.location(), 0L), spec, null); writer2.close(); // verify the writer result diff --git a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java index 1e5863e8bff9..a4122d593c28 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java @@ -164,8 +164,9 @@ public void testPositionDeltaDeleteOnly() throws IOException { PositionDeltaWriter deltaWriter = new BasePositionDeltaWriter<>(insertWriter, updateWriter, deleteWriter); - deltaWriter.delete(dataFile1.path(), 2L, unpartitionedSpec, null); - deltaWriter.delete(dataFile2.path(), 1L, partitionedSpec, partitionKey(partitionedSpec, "bbb")); + deltaWriter.delete(dataFile1.location(), 2L, unpartitionedSpec, null); + deltaWriter.delete( + dataFile2.location(), 1L, partitionedSpec, partitionKey(partitionedSpec, "bbb")); deltaWriter.close(); @@ -219,8 +220,9 @@ public void testPositionDeltaMultipleSpecs() throws IOException { PositionDeltaWriter deltaWriter = new BasePositionDeltaWriter<>(insertWriter, updateWriter, deleteWriter); - deltaWriter.delete(dataFile1.path(), 2L, unpartitionedSpec, null); - deltaWriter.delete(dataFile2.path(), 1L, partitionedSpec, partitionKey(partitionedSpec, "bbb")); + deltaWriter.delete(dataFile1.location(), 2L, unpartitionedSpec, null); + deltaWriter.delete( + dataFile2.location(), 1L, partitionedSpec, partitionKey(partitionedSpec, "bbb")); deltaWriter.insert(toRow(10, "ccc"), partitionedSpec, partitionKey(partitionedSpec, "ccc")); deltaWriter.close(); diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 01a998c65efe..b6b15e766835 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -464,8 +464,8 @@ private void checkDataFilePathsIntegrity( .addedDataFiles(icebergTable.io()) .forEach( dataFile -> { - assertThat(URI.create(dataFile.path().toString()).isAbsolute()).isTrue(); - assertThat(deltaTableDataFilePaths).contains(dataFile.path().toString()); + assertThat(URI.create(dataFile.location()).isAbsolute()).isTrue(); + assertThat(deltaTableDataFilePaths).contains(dataFile.location()); }); } diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index afe43f6bb165..20c6976a4a24 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -252,7 +252,7 @@ private long commitInitialDeltaSnapshotToIcebergTransaction( for (AddFile addFile : initDataFiles) { DataFile dataFile = buildDataFileFromAction(addFile, transaction.table()); filesToAdd.add(dataFile); - migratedDataFilesBuilder.add(dataFile.path().toString()); + migratedDataFilesBuilder.add(dataFile.location()); } // AppendFiles case @@ -309,7 +309,7 @@ private void commitDeltaVersionLogToIcebergTransaction( throw new ValidationException( "The action %s's is unsupported", action.getClass().getSimpleName()); } - migratedDataFilesBuilder.add(dataFile.path().toString()); + migratedDataFilesBuilder.add(dataFile.location()); } if (!filesToAdd.isEmpty() && !filesToRemove.isEmpty()) { diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 0f24c5613f17..34fec09add7c 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -283,7 +283,7 @@ public void testExpireDeleteFiles() throws Exception { assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); + new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().location())); sql( "CALL %s.system.rewrite_data_files(" diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index d8feaa77079b..0928cde7ca66 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -412,7 +412,7 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); + new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().location())); // wait to ensure files are old enough waitUntilAfter(System.currentTimeMillis()); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java index f7329e841800..f3be0a870972 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -305,7 +305,7 @@ private void writePosDeletesForFiles(Table table, List files) throws I List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < DELETE_FILE_SIZE; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java index 09aa51f0460a..550bf41ce220 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -1299,7 +1299,7 @@ public void testUpdateWithStaticPredicatePushdown() { // remove the data file from the 'hr' partition to ensure it is not scanned DataFile dataFile = Iterables.getOnlyElement(snapshot.addedDataFiles(table.io())); - table.io().deleteFile(dataFile.path().toString()); + table.io().deleteFile(dataFile.location()); // disable dynamic pruning and rely only on static predicate pushdown withSQLConf( diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index 53ce7418f3ec..34bb4afe67f9 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -432,7 +432,7 @@ public CloseableIterator entries(ManifestFileBean manifest) { } static FileInfo toFileInfo(ContentFile file) { - return new FileInfo(file.path().toString(), file.content().toString()); + return new FileInfo(file.location(), file.content().toString()); } } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java index bbf65f58e19c..1474ec0e3eef 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java @@ -90,7 +90,7 @@ Result doExecute() { RewriteFiles rewriteFiles = table.newRewrite(); List danglingDeletes = findDanglingDeletes(); for (DeleteFile deleteFile : danglingDeletes) { - LOG.debug("Removing dangling delete file {}", deleteFile.path()); + LOG.debug("Removing dangling delete file {}", deleteFile.location()); rewriteFiles.deleteFile(deleteFile); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index 207035bd30ac..f8e8a1f1dd6b 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -150,7 +150,7 @@ public boolean next() throws IOException { if (currentTask != null && !currentTask.isDataTask()) { String filePaths = referencedFiles(currentTask) - .map(file -> file.path().toString()) + .map(ContentFile::location) .collect(Collectors.joining(", ")); LOG.error("Error reading file(s): {}", filePaths, e); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index a2cb74c926c9..f45c152203ee 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -82,7 +82,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); // update the current file for Spark's filename() function diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index 25cd9eda6bce..c8e6f5679cd8 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -110,7 +110,7 @@ private CloseableIterable openChangelogScanTask(ChangelogScanTask t } CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java index 7aa5a971562a..ee9449ee13c8 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java @@ -41,7 +41,7 @@ public EqualityDeleteRowReader( @Override protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter matches = - new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter(), true); + new SparkDeleteFilter(task.file().location(), task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = matches.requiredSchema(); @@ -49,7 +49,7 @@ protected CloseableIterator open(FileScanTask task) { DataFile file = task.file(); // update the current file for Spark's filename() function - InputFileBlockHolder.set(file.path().toString(), task.start(), task.length()); + InputFileBlockHolder.set(file.location(), task.start(), task.length()); return matches.findEqualityDeleteRows(open(task, requiredSchema, idToConstant)).iterator(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java index 4b847474153c..1a894df29166 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java @@ -74,13 +74,13 @@ protected Stream> referencedFiles(PositionDeletesScanTask task) { @Override protected CloseableIterator open(PositionDeletesScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening position delete file {}", filePath); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with %s", task); // select out constant fields when pushing down filter to row reader diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 33b1d6275d6e..f24602fd5583 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -81,7 +81,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); SparkDeleteFilter deleteFilter = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); @@ -101,7 +101,7 @@ protected CloseableIterable open( if (task.isDataTask()) { return newDataIterable(task.asDataTask(), readSchema); } else { - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull( inputFile, "Could not find InputFile associated with FileScanTask"); return newIterable( diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java index a103a5003222..5f343128161d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java @@ -81,7 +81,7 @@ private static String taskInfo() { * @param files a list of files to delete */ public static void deleteFiles(String context, FileIO io, List> files) { - List paths = Lists.transform(files, file -> file.path().toString()); + List paths = Lists.transform(files, ContentFile::location); deletePaths(context, io, paths); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index 5a3963bc614c..7a6025b0731a 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -130,7 +130,7 @@ public void filter(Filter[] filters) { this.filteredLocations = fileLocations; List filteredTasks = tasks().stream() - .filter(file -> fileLocations.contains(file.file().path().toString())) + .filter(file -> fileLocations.contains(file.file().location())) .collect(Collectors.toList()); LOG.info( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java index 668f410091dd..bcd00eb6f4e5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java @@ -62,9 +62,9 @@ public static void assertEquals(FileScanTask expected, FileScanTask actual) { } public static void assertEquals(DataFile expected, DataFile actual) { - assertThat(actual.path()) + assertThat(actual.location()) .as("Should match the serialized record path") - .isEqualTo(expected.path()); + .isEqualTo(expected.location()); assertThat(actual.format()) .as("Should match the serialized record format") .isEqualTo(expected.format()); @@ -104,7 +104,7 @@ private static List getFileScanTasksInFilePathOrder( ScanTaskGroup taskGroup) { return taskGroup.tasks().stream() // use file path + start position to differentiate the tasks - .sorted(Comparator.comparing(o -> o.file().path().toString() + "##" + o.start())) + .sorted(Comparator.comparing(o -> o.file().location() + "##" + o.start())) .collect(Collectors.toList()); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java index b0b3085dca70..7314043b15e2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java @@ -42,7 +42,7 @@ public static List snapshotIds(Long... ids) { } public static List files(ContentFile... files) { - return Arrays.stream(files).map(file -> file.path().toString()).collect(Collectors.toList()); + return Arrays.stream(files).map(file -> file.location()).collect(Collectors.toList()); } public static void validateDataManifest( @@ -62,7 +62,7 @@ public static void validateDataManifest( actualDataSeqs.add(entry.dataSequenceNumber()); actualFileSeqs.add(entry.fileSequenceNumber()); actualSnapshotIds.add(entry.snapshotId()); - actualFiles.add(entry.file().path().toString()); + actualFiles.add(entry.file().location()); } assertSameElements("data seqs", actualDataSeqs, dataSeqs); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java index 40bc4bbce54e..86afd2f890ae 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java @@ -142,7 +142,7 @@ protected static String dbPath(String dbName) { } protected void withUnavailableFiles(Iterable> files, Action action) { - Iterable fileLocations = Iterables.transform(files, file -> file.path().toString()); + Iterable fileLocations = Iterables.transform(files, file -> file.location()); withUnavailableLocations(fileLocations, action); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java index c5463b4249d6..d9d7b78799ba 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java @@ -350,7 +350,7 @@ private void checkMerge(RowLevelOperationMode mode) throws Exception { } private int streamCount(DeleteFile deleteFile) { - CustomInputFile inputFile = INPUT_FILES.get(deleteFile.path().toString()); + CustomInputFile inputFile = INPUT_FILES.get(deleteFile.location()); return inputFile.streamCount(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index ad93b80baf81..d5bb63b2d88a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -220,7 +220,7 @@ public void dataFilesCleanupWithParallelTasks() { file -> assertThat(deletedFiles) .as("FILE_A should be deleted") - .contains(FILE_A.path().toString())); + .contains(FILE_A.location())); checkRemoveFilesResults(4L, 0, 0, 6L, 4L, 6, result); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 661df99ef3e6..ffbe988e8d41 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -261,8 +261,8 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { "remove-snapshot-2", "remove-snapshot-3")); - assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.path().toString()); - assertThat(deletedFiles).as("FILE_B should be deleted").contains(FILE_B.path().toString()); + assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.location()); + assertThat(deletedFiles).as("FILE_B should be deleted").contains(FILE_B.location()); checkExpirationResults(2L, 0L, 0L, 3L, 3L, result); } @@ -565,7 +565,7 @@ public void testScanExpiredManifestInValidSnapshotAppend() { .deleteWith(deletedFiles::add) .execute(); - assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.path().toString()); + assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.location()); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -594,7 +594,7 @@ public void testScanExpiredManifestInValidSnapshotFastAppend() { .deleteWith(deletedFiles::add) .execute(); - assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.path().toString()); + assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.location()); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -637,7 +637,7 @@ public void testWithExpiringDanglingStageCommit() { .addedDataFiles(table.io()) .forEach( i -> { - expectedDeletes.add(i.path().toString()); + expectedDeletes.add(i.location()); }); // ManifestList should be deleted too @@ -707,7 +707,7 @@ public void testWithCherryPickTableSnapshot() { i.addedDataFiles(table.io()) .forEach( item -> { - assertThat(deletedFiles).doesNotContain(item.path().toString()); + assertThat(deletedFiles).doesNotContain(item.location()); }); }); @@ -756,7 +756,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - assertThat(deletedFiles).doesNotContain(item.path().toString()); + assertThat(deletedFiles).doesNotContain(item.location()); }); }); checkExpirationResults(0L, 0L, 0L, 1L, 1L, firstResult); @@ -776,7 +776,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - assertThat(deletedFiles).doesNotContain(item.path().toString()); + assertThat(deletedFiles).doesNotContain(item.location()); }); }); checkExpirationResults(0L, 0L, 0L, 0L, 2L, secondResult); @@ -1100,9 +1100,9 @@ public void testExpireOlderThanWithDeleteFile() { secondSnapshot.manifestListLocation(), thirdSnapshot.manifestListLocation(), fourthSnapshot.manifestListLocation(), - FILE_A.path().toString(), - fileADeletes.path().toString(), - FILE_A_EQ_DELETES.path().toString()); + FILE_A.location(), + fileADeletes.location(), + FILE_A_EQ_DELETES.location()); expectedDeletes.addAll( thirdSnapshot.allManifests(table.io()).stream() @@ -1277,7 +1277,7 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe .withPartitionPath("c1=1") .withRecordCount(1) .build(); - dataFiles.add(df.path().toString()); + dataFiles.add(df.location()); table.newFastAppend().appendFile(df).commit(); } @@ -1351,9 +1351,9 @@ public void testExpireSomeCheckFilesDeleted() { // C, D should be retained (live) // B should be retained (previous snapshot points to it) // A should be deleted - assertThat(deletedFiles).contains(FILE_A.path().toString()); - assertThat(deletedFiles).doesNotContain(FILE_B.path().toString()); - assertThat(deletedFiles).doesNotContain(FILE_C.path().toString()); - assertThat(deletedFiles).doesNotContain(FILE_D.path().toString()); + assertThat(deletedFiles).contains(FILE_A.location()); + assertThat(deletedFiles).doesNotContain(FILE_B.location()); + assertThat(deletedFiles).doesNotContain(FILE_C.location()); + assertThat(deletedFiles).doesNotContain(FILE_D.location()); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java index 7258344823f7..a6d7d4827c0d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java @@ -114,7 +114,7 @@ public Boolean hasIterator(FileScanTask task) { } private String getKey(FileScanTask task) { - return task.file().path().toString(); + return task.file().location(); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java index 217b05b46a7c..f411920a5dcc 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java @@ -209,7 +209,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { List manifestFiles = table.currentSnapshot().dataManifests(table.io()); try (ManifestReader reader = ManifestFiles.read(manifestFiles.get(0), table.io())) { DataFile file = reader.iterator().next(); - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -223,7 +223,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { try (ManifestReader reader = ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -237,7 +237,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { try (ManifestReader reader = ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index a8b4c915868a..42552f385137 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -187,11 +187,11 @@ private void writeAndValidateWithLocations(Table table, File expectedDataDir) th .addedDataFiles(table.io()) .forEach( dataFile -> - assertThat(URI.create(dataFile.path().toString()).getPath()) + assertThat(URI.create(dataFile.location()).getPath()) .as( String.format( "File should have the parent directory %s, but has: %s.", - expectedDataDir.getAbsolutePath(), dataFile.path())) + expectedDataDir.getAbsolutePath(), dataFile.location())) .startsWith(expectedDataDir.getAbsolutePath())); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 627fe15f2819..c4ba96e63403 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -109,7 +109,7 @@ public void testWriteFormatOptionOverridesTableProperties() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); assertThat(fileFormat).isEqualTo(FileFormat.PARQUET); }); } @@ -134,7 +134,7 @@ public void testNoWriteFormatOption() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); assertThat(fileFormat).isEqualTo(FileFormat.AVRO); }); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java index ca934772f6af..983e52cdf897 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -134,8 +134,8 @@ public void testNullRows() throws IOException { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, @@ -147,9 +147,9 @@ public void testNullRows() throws IOException { StructLikeSet actual = actual(tableName, tab); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); - StructLikeSet expected = - expected(tab, expectedDeletes, null, posDeletes.first().path().toString()); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); + StructLikeSet expected = expected(tab, expectedDeletes, null, posDeletes.first().location()); assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); dropTable(tableName); @@ -178,7 +178,7 @@ public void testPartitionedTable() throws IOException { GenericRecord partitionB = GenericRecord.create(tab.spec().partitionType()); partitionB.setField("data", "b"); StructLikeSet expected = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); dropTable(tableName); @@ -218,7 +218,7 @@ public void testSelect() throws IOException { (delete, file) -> { int rowData = delete.get(2, GenericRecord.class).get(0, Integer.class); long pos = delete.get(1, Long.class); - return row(rowData, pos, file.path().toString(), file.path().toString()); + return row(rowData, pos, file.location(), file.location()); }; expected.addAll( deletesA.first().stream() @@ -270,7 +270,7 @@ public void testSplitTasks() throws IOException { List> deletes = Lists.newArrayList(); for (long i = 0; i < records; i++) { - deletes.add(positionDelete(tab.schema(), dFile.path(), i, (int) i, String.valueOf(i))); + deletes.add(positionDelete(tab.schema(), dFile.location(), i, (int) i, String.valueOf(i))); } DeleteFile posDeletes = FileHelpers.writePosDeleteFile( @@ -294,7 +294,7 @@ public void testSplitTasks() throws IOException { } StructLikeSet actual = actual(tableName, tab); - StructLikeSet expected = expected(tab, deletes, null, posDeletes.path().toString()); + StructLikeSet expected = expected(tab, deletes, null, posDeletes.location()); assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); dropTable(tableName); @@ -324,9 +324,9 @@ public void testPartitionFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data", "a"); Record partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTab.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -371,9 +371,9 @@ public void testPartitionTransformFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data_trunc", "a"); Record partitionB = partitionRecordTemplate.copy("data_trunc", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTable.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -425,7 +425,7 @@ public void testPartitionEvolutionReplace() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); assertThat(actualA) .as("Position Delete table should contain expected rows") @@ -439,7 +439,7 @@ public void testPartitionEvolutionReplace() throws Exception { deletes10.first(), partition10, tab.spec().specId(), - deletes10.second().path().toString()); + deletes10.second().location()); StructLikeSet actual10 = actual(tableName, tab, "partition.id = 10 AND pos >= 0"); assertThat(actual10) @@ -479,7 +479,7 @@ public void testPartitionEvolutionAdd() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); assertThat(actualA) .as("Position Delete table should contain expected rows") @@ -493,7 +493,7 @@ public void testPartitionEvolutionAdd() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId0, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -535,7 +535,7 @@ public void testPartitionEvolutionRemove() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); assertThat(actualA) .as("Position Delete table should contain expected rows") @@ -549,7 +549,7 @@ public void testPartitionEvolutionRemove() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId1, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -593,7 +593,7 @@ public void testSpecIdFilter() throws Exception { deletesUnpartitioned.first(), partitionRecordTemplate, unpartitionedSpec, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, String.format("spec_id = %d", unpartitionedSpec)); assertThat(actualUnpartitioned) @@ -604,9 +604,9 @@ public void testSpecIdFilter() throws Exception { StructLike partitionA = partitionRecordTemplate.copy("data", "a"); StructLike partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expected = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); expected.addAll( - expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().path().toString())); + expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().location())); StructLikeSet actual = actual(tableName, tab, String.format("spec_id = %d", dataSpec)); assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); @@ -660,7 +660,7 @@ public void testSchemaEvolutionAdd() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); assertThat(actualA) .as("Position Delete table should contain expected rows") @@ -669,7 +669,7 @@ public void testSchemaEvolutionAdd() throws Exception { // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); assertThat(actualC) @@ -726,7 +726,7 @@ public void testSchemaEvolutionRemove() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); assertThat(actualA) .as("Position Delete table should contain expected rows") @@ -735,7 +735,7 @@ public void testSchemaEvolutionRemove() throws Exception { // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); assertThat(actualC) @@ -817,8 +817,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, @@ -855,7 +855,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { actual(tableName, tab, null, ImmutableList.of("file_path", "pos", "row", "spec_id")); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); StructLikeSet expected = expected(tab, expectedDeletes, null, null); assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); @@ -874,8 +875,8 @@ public void testWriteMixedRows() throws Exception { // Add a delete file with row and without row List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dataFileA.path(), 0L)); - deletes.add(Pair.of(dataFileA.path(), 1L)); + deletes.add(Pair.of(dataFileA.location(), 0L)); + deletes.add(Pair.of(dataFileA.location(), 1L)); Pair deletesWithoutRow = FileHelpers.writeDeleteFile( tab, @@ -936,7 +937,7 @@ public void testWriteMixedRows() throws Exception { expected( tab, Lists.newArrayList( - positionDelete(dataFileA.path(), 0L), positionDelete(dataFileA.path(), 1L)), + positionDelete(dataFileA.location(), 0L), positionDelete(dataFileA.location(), 1L)), partitionA, null)); allExpected.addAll(expected(tab, deletesWithRow.first(), partitionB, null)); @@ -1530,13 +1531,13 @@ private Pair>, DeleteFile> deleteFile( Lists.newArrayList( positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 0L, idPartition != null ? idPartition : 29, dataPartition != null ? dataPartition : "c"), positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 1L, idPartition != null ? idPartition : 61, dataPartition != null ? dataPartition : "r")); @@ -1585,9 +1586,9 @@ private void commit( assertThat(addedFiles).hasSize(expectedTargetFiles); List sortedAddedFiles = - addedFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + addedFiles.stream().map(f -> f.location()).sorted().collect(Collectors.toList()); List sortedRewrittenFiles = - rewrittenFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + rewrittenFiles.stream().map(f -> f.location()).sorted().collect(Collectors.toList()); assertThat(sortedRewrittenFiles) .as("Lists should not be the same") .isNotEqualTo(sortedAddedFiles); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java index 21085b49df23..e7346e270f38 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java @@ -483,7 +483,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set matchingFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().filter(filter).planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); matchingFileLocations.add(path); } } catch (IOException e) { @@ -493,7 +493,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set deletedFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); if (!matchingFileLocations.contains(path)) { io.deleteFile(path); deletedFileLocations.add(path); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index c9604863b9fa..182b1ef8f5af 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -279,7 +279,7 @@ private void checkDeleteFile(DeleteFile expected, DeleteFile actual) { private void checkContentFile(ContentFile expected, ContentFile actual) { assertThat(actual.content()).isEqualTo(expected.content()); - assertThat(actual.path()).isEqualTo(expected.path()); + assertThat(actual.location()).isEqualTo(expected.location()); assertThat(actual.format()).isEqualTo(expected.format()); assertThat(actual.recordCount()).isEqualTo(expected.recordCount()); assertThat(actual.fileSizeInBytes()).isEqualTo(expected.fileSizeInBytes()); @@ -317,10 +317,10 @@ private DeleteFile createPositionDeleteFile(Table table, DataFile dataFile) { null, // no NaN counts ImmutableMap.of( MetadataColumns.DELETE_FILE_PATH.fieldId(), - Conversions.toByteBuffer(Types.StringType.get(), dataFile.path())), + Conversions.toByteBuffer(Types.StringType.get(), dataFile.location())), ImmutableMap.of( MetadataColumns.DELETE_FILE_PATH.fieldId(), - Conversions.toByteBuffer(Types.StringType.get(), dataFile.path())))) + Conversions.toByteBuffer(Types.StringType.get(), dataFile.location())))) .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) .build(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index d49b450524d9..d1ed1dc2b3cf 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -343,10 +343,10 @@ public void testPosDeletesAllRowsInBatch() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -376,10 +376,10 @@ public void testPosDeletesWithDeletedColumn() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -455,8 +455,8 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 3L), // id = 89 - Pair.of(dataFile.path(), 5L) // id = 121 + Pair.of(dataFile.location(), 3L), // id = 89 + Pair.of(dataFile.location(), 5L) // id = 121 ); Pair posDeletes = @@ -486,10 +486,10 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { public void testFilterOnDeletedMetadataColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -611,13 +611,13 @@ public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOExceptio // Add positional deletes to the table List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 97L), - Pair.of(dataFile.path(), 98L), - Pair.of(dataFile.path(), 99L), - Pair.of(dataFile.path(), 101L), - Pair.of(dataFile.path(), 103L), - Pair.of(dataFile.path(), 107L), - Pair.of(dataFile.path(), 109L)); + Pair.of(dataFile.location(), 97L), + Pair.of(dataFile.location(), 98L), + Pair.of(dataFile.location(), 99L), + Pair.of(dataFile.location(), 101L), + Pair.of(dataFile.location(), 103L), + Pair.of(dataFile.location(), 107L), + Pair.of(dataFile.location(), 109L)); Pair posDeletes = FileHelpers.writeDeleteFile( table, From bf8d25fe1578ef199d64fb609c0299728ec58910 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 18 Nov 2024 18:14:35 +0100 Subject: [PATCH 156/313] Core: Serialize `null` when there is no current snapshot (#11560) --- .../main/java/org/apache/iceberg/TableMetadataParser.java | 8 +++++--- .../rest/responses/TestLoadTableResponseParser.java | 6 +++--- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index 5abfb74974f7..c244b3996c9e 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -214,9 +214,11 @@ public static void toJson(TableMetadata metadata, JsonGenerator generator) throw // write properties map JsonUtil.writeStringMap(PROPERTIES, metadata.properties(), generator); - generator.writeNumberField( - CURRENT_SNAPSHOT_ID, - metadata.currentSnapshot() != null ? metadata.currentSnapshot().snapshotId() : -1); + if (metadata.currentSnapshot() != null) { + generator.writeNumberField(CURRENT_SNAPSHOT_ID, metadata.currentSnapshot().snapshotId()); + } else { + generator.writeNullField(CURRENT_SNAPSHOT_ID); + } toJson(metadata.refs(), generator); diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java index cc6f4cfc74d7..b4879dd39f51 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java @@ -107,7 +107,7 @@ public void roundTripSerde() { + " \"fields\" : [ ]\n" + " } ],\n" + " \"properties\" : { },\n" - + " \"current-snapshot-id\" : -1,\n" + + " \"current-snapshot-id\" : null,\n" + " \"refs\" : { },\n" + " \"snapshots\" : [ ],\n" + " \"statistics\" : [ ],\n" @@ -180,7 +180,7 @@ public void roundTripSerdeWithConfig() { + " \"fields\" : [ ]\n" + " } ],\n" + " \"properties\" : { },\n" - + " \"current-snapshot-id\" : -1,\n" + + " \"current-snapshot-id\" : null,\n" + " \"refs\" : { },\n" + " \"snapshots\" : [ ],\n" + " \"statistics\" : [ ],\n" @@ -283,7 +283,7 @@ public void roundTripSerdeWithCredentials() { + " \"fields\" : [ ]\n" + " } ],\n" + " \"properties\" : { },\n" - + " \"current-snapshot-id\" : -1,\n" + + " \"current-snapshot-id\" : null,\n" + " \"refs\" : { },\n" + " \"snapshots\" : [ ],\n" + " \"statistics\" : [ ],\n" From 209781af4f48cbde23a864ae6da98022eccb4708 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 19 Nov 2024 04:11:17 +0800 Subject: [PATCH 157/313] Spark 3.4: Iceberg parser should passthrough unsupported procedure to delegate (#11579) --- .../IcebergSparkSqlExtensionsParser.scala | 11 ++- .../extensions/TestCallStatementParser.java | 78 ++++++++++++++----- .../TestCherrypickSnapshotProcedure.java | 12 ++- .../TestExpireSnapshotsProcedure.java | 11 ++- .../TestFastForwardBranchProcedure.java | 12 ++- .../TestPublishChangesProcedure.java | 12 ++- .../TestRemoveOrphanFilesProcedure.java | 10 ++- .../TestRewriteDataFilesProcedure.java | 11 ++- .../TestRewriteManifestsProcedure.java | 11 ++- .../TestRollbackToSnapshotProcedure.java | 12 ++- .../TestRollbackToTimestampProcedure.java | 12 ++- .../TestSetCurrentSnapshotProcedure.java | 12 ++- .../spark/procedures/SparkProcedures.java | 5 ++ 13 files changed, 157 insertions(+), 52 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index 1f74f2f811c8..f29e31efba41 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -29,6 +29,7 @@ import org.apache.iceberg.common.DynConstructors import org.apache.iceberg.spark.ExtendedParser import org.apache.iceberg.spark.ExtendedParser.RawOrderField import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.procedures.SparkProcedures import org.apache.iceberg.spark.source.SparkTable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.SparkSession @@ -194,8 +195,10 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI // Strip comments of the form /* ... */. This must come after stripping newlines so that // comments that span multiple lines are caught. .replaceAll("/\\*.*?\\*/", " ") + // Strip backtick then `system`.`ancestors_of` changes to system.ancestors_of + .replaceAll("`", "") .trim() - normalized.startsWith("call") || ( + isIcebergProcedure(normalized) || ( normalized.startsWith("alter table") && ( normalized.contains("add partition field") || normalized.contains("drop partition field") || @@ -209,6 +212,12 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI isSnapshotRefDdl(normalized))) } + // All builtin Iceberg procedures are under the 'system' namespace + private def isIcebergProcedure(normalized: String): Boolean = { + normalized.startsWith("call") && + SparkProcedures.names().asScala.map("system." + _).exists(normalized.contains) + } + private def isSnapshotRefDdl(normalized: String): Boolean = { normalized.contains("create branch") || normalized.contains("replace branch") || diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java index a4630ab279e5..55a413063eec 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java @@ -73,12 +73,37 @@ public static void stopSpark() { currentSpark.stop(); } + @Test + public void testDelegateUnsupportedProcedure() { + assertThatThrownBy(() -> parser.parsePlan("CALL cat.d.t()")) + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); + } + + @Test + public void testCallWithBackticks() throws ParseException { + CallStatement call = + (CallStatement) parser.parsePlan("CALL cat.`system`.`rollback_to_snapshot`()"); + Assert.assertEquals( + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); + Assert.assertEquals(0, call.args().size()); + } + @Test public void testCallWithPositionalArgs() throws ParseException { CallStatement call = - (CallStatement) parser.parsePlan("CALL c.n.func(1, '2', 3L, true, 1.0D, 9.0e1, 900e-1BD)"); + (CallStatement) + parser.parsePlan( + "CALL c.system.rollback_to_snapshot(1, '2', 3L, true, 1.0D, 9.0e1, 900e-1BD)"); Assert.assertEquals( - ImmutableList.of("c", "n", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("c", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(7, call.args().size()); @@ -94,9 +119,12 @@ public void testCallWithPositionalArgs() throws ParseException { @Test public void testCallWithNamedArgs() throws ParseException { CallStatement call = - (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, c2 => '2', c3 => true)"); + (CallStatement) + parser.parsePlan( + "CALL cat.system.rollback_to_snapshot(c1 => 1, c2 => '2', c3 => true)"); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(3, call.args().size()); @@ -107,9 +135,11 @@ public void testCallWithNamedArgs() throws ParseException { @Test public void testCallWithMixedArgs() throws ParseException { - CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, '2')"); + CallStatement call = + (CallStatement) parser.parsePlan("CALL cat.system.rollback_to_snapshot(c1 => 1, '2')"); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(2, call.args().size()); @@ -121,9 +151,11 @@ public void testCallWithMixedArgs() throws ParseException { public void testCallWithTimestampArg() throws ParseException { CallStatement call = (CallStatement) - parser.parsePlan("CALL cat.system.func(TIMESTAMP '2017-02-03T10:37:30.00Z')"); + parser.parsePlan( + "CALL cat.system.rollback_to_snapshot(TIMESTAMP '2017-02-03T10:37:30.00Z')"); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(1, call.args().size()); @@ -134,9 +166,11 @@ public void testCallWithTimestampArg() throws ParseException { @Test public void testCallWithVarSubstitution() throws ParseException { CallStatement call = - (CallStatement) parser.parsePlan("CALL cat.system.func('${spark.extra.prop}')"); + (CallStatement) + parser.parsePlan("CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')"); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(1, call.args().size()); @@ -145,29 +179,31 @@ public void testCallWithVarSubstitution() throws ParseException { @Test public void testCallParseError() { - assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) + assertThatThrownBy(() -> parser.parsePlan("CALL cat.system.rollback_to_snapshot kebab")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("missing '(' at 'radish'"); + .hasMessageContaining("missing '(' at 'kebab'"); } @Test public void testCallStripsComments() throws ParseException { List callStatementsWithComments = Lists.newArrayList( - "/* bracketed comment */ CALL cat.system.func('${spark.extra.prop}')", - "/**/ CALL cat.system.func('${spark.extra.prop}')", - "-- single line comment \n CALL cat.system.func('${spark.extra.prop}')", - "-- multiple \n-- single line \n-- comments \n CALL cat.system.func('${spark.extra.prop}')", - "/* select * from multiline_comment \n where x like '%sql%'; */ CALL cat.system.func('${spark.extra.prop}')", + "/* bracketed comment */ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "/**/ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "-- single line comment \n CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "-- multiple \n-- single line \n-- comments \n CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "/* select * from multiline_comment \n where x like '%sql%'; */ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", "/* {\"app\": \"dbt\", \"dbt_version\": \"1.0.1\", \"profile_name\": \"profile1\", \"target_name\": \"dev\", " - + "\"node_id\": \"model.profile1.stg_users\"} \n*/ CALL cat.system.func('${spark.extra.prop}')", + + "\"node_id\": \"model.profile1.stg_users\"} \n*/ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", "/* Some multi-line comment \n" - + "*/ CALL /* inline comment */ cat.system.func('${spark.extra.prop}') -- ending comment", - "CALL -- a line ending comment\n" + "cat.system.func('${spark.extra.prop}')"); + + "*/ CALL /* inline comment */ cat.system.rollback_to_snapshot('${spark.extra.prop}') -- ending comment", + "CALL -- a line ending comment\n" + + "cat.system.rollback_to_snapshot('${spark.extra.prop}')"); for (String sqlText : callStatementsWithComments) { CallStatement call = (CallStatement) parser.parsePlan(sqlText); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(1, call.args().size()); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java index 51762335a03b..b956db3d5512 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java @@ -31,8 +31,9 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Test; public class TestCherrypickSnapshotProcedure extends SparkExtensionsTestBase { @@ -175,8 +176,13 @@ public void testInvalidCherrypickSnapshotCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.cherrypick_snapshot not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 913d70f1ca91..8235aca7f764 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -52,7 +52,7 @@ import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -175,8 +175,13 @@ public void testInvalidExpireSnapshotsCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.expire_snapshots not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java index 37d40957bc44..fea147b3e61f 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java @@ -30,8 +30,9 @@ import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Test; public class TestFastForwardBranchProcedure extends SparkExtensionsTestBase { @@ -176,8 +177,13 @@ public void testInvalidFastForwardBranchCases() { assertThatThrownBy( () -> sql("CALL %s.custom.fast_forward('test_table', 'main', 'newBranch')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.fast_forward not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.fast_forward('test_table', 'main')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java index fe0f12a9d0bb..6bf1beec1eac 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java @@ -31,8 +31,9 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Test; public class TestPublishChangesProcedure extends SparkExtensionsTestBase { @@ -173,8 +174,13 @@ public void testInvalidApplyWapChangesCases() { assertThatThrownBy( () -> sql("CALL %s.custom.publish_changes('n', 't', 'not_valid')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.publish_changes not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index b7f82725aab8..3b1ca1b239b8 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -63,7 +63,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; @@ -264,8 +263,13 @@ public void testInvalidRemoveOrphanFilesCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.remove_orphan_files not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 24230d80b08d..e637950ae5d4 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -41,7 +41,7 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.internal.SQLConf; import org.junit.After; import org.junit.Assert; @@ -697,8 +697,13 @@ public void testInvalidCasesForRewriteDataFiles() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.rewrite_data_files not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 3b8650de3454..4feb3ae8e49c 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -30,8 +30,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -282,8 +282,13 @@ public void testInvalidRewriteManifestsCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.rewrite_manifests not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java index 3e9b324df75d..ba57f6a6823f 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java @@ -29,8 +29,9 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -258,8 +259,13 @@ public void testInvalidRollbackToSnapshotCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.rollback_to_snapshot not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java index 1cfe1ecf83be..f38e58199843 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java @@ -31,8 +31,9 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -301,8 +302,13 @@ public void testInvalidRollbackToTimestampCases() { assertThatThrownBy( () -> sql("CALL %s.custom.rollback_to_timestamp('n', 't', %s)", catalogName, timestamp)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.rollback_to_timestamp not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java index 65db75f6086d..0232e80da1bc 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java @@ -29,8 +29,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.spark.sql.AnalysisException; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -206,8 +207,13 @@ public void testInvalidRollbackToSnapshotCases() { .hasMessage("Named and positional arguments cannot be mixed"); assertThatThrownBy(() -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.set_current_snapshot not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("'CALL'", parseException.getMessageParameters().get("error")); + }); assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) .isInstanceOf(IllegalArgumentException.class) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java index b324cd4422b3..42003b24e94c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java @@ -20,6 +20,7 @@ import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.function.Supplier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -37,6 +38,10 @@ public static ProcedureBuilder newBuilder(String name) { return builderSupplier != null ? builderSupplier.get() : null; } + public static Set names() { + return BUILDERS.keySet(); + } + private static Map> initProcedureBuilders() { ImmutableMap.Builder> mapBuilder = ImmutableMap.builder(); mapBuilder.put("rollback_to_snapshot", RollbackToSnapshotProcedure::builder); From 568940f5f85568868f3d75855fa72d17378271f3 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 19 Nov 2024 04:11:41 +0800 Subject: [PATCH 158/313] Spark 3.3: Iceberg parser should passthrough unsupported procedure to delegate (#11580) --- .../IcebergSparkSqlExtensionsParser.scala | 11 ++- .../extensions/TestCallStatementParser.java | 78 ++++++++++++++----- .../TestCherrypickSnapshotProcedure.java | 12 ++- .../TestExpireSnapshotsProcedure.java | 10 ++- .../TestFastForwardBranchProcedure.java | 12 ++- .../TestPublishChangesProcedure.java | 11 ++- .../TestRemoveOrphanFilesProcedure.java | 9 ++- .../TestRewriteDataFilesProcedure.java | 10 ++- .../TestRewriteManifestsProcedure.java | 10 ++- .../TestRollbackToSnapshotProcedure.java | 11 ++- .../TestRollbackToTimestampProcedure.java | 11 ++- .../TestSetCurrentSnapshotProcedure.java | 11 ++- .../spark/procedures/SparkProcedures.java | 5 ++ 13 files changed, 149 insertions(+), 52 deletions(-) diff --git a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index 7c17ea667e0b..b281cfedb7bc 100644 --- a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -29,6 +29,7 @@ import org.apache.iceberg.common.DynConstructors import org.apache.iceberg.spark.ExtendedParser import org.apache.iceberg.spark.ExtendedParser.RawOrderField import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.procedures.SparkProcedures import org.apache.iceberg.spark.source.SparkTable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.SparkSession @@ -194,8 +195,10 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI // Strip comments of the form /* ... */. This must come after stripping newlines so that // comments that span multiple lines are caught. .replaceAll("/\\*.*?\\*/", " ") + // Strip backtick then `system`.`ancestors_of` changes to system.ancestors_of + .replaceAll("`", "") .trim() - normalized.startsWith("call") || ( + isIcebergProcedure(normalized) || ( normalized.startsWith("alter table") && ( normalized.contains("add partition field") || normalized.contains("drop partition field") || @@ -209,6 +212,12 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI isSnapshotRefDdl(normalized))) } + // All builtin Iceberg procedures are under the 'system' namespace + private def isIcebergProcedure(normalized: String): Boolean = { + normalized.startsWith("call") && + SparkProcedures.names().asScala.map("system." + _).exists(normalized.contains) + } + private def isSnapshotRefDdl(normalized: String): Boolean = { normalized.contains("create branch") || normalized.contains("replace branch") || diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java index 42f29c28c149..0e5948028bb3 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCallStatementParser.java @@ -73,12 +73,37 @@ public static void stopSpark() { currentSpark.stop(); } + @Test + public void testDelegateUnsupportedProcedure() { + assertThatThrownBy(() -> parser.parsePlan("CALL cat.d.t()")) + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); + } + + @Test + public void testCallWithBackticks() throws ParseException { + CallStatement call = + (CallStatement) parser.parsePlan("CALL cat.`system`.`rollback_to_snapshot`()"); + Assert.assertEquals( + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); + Assert.assertEquals(0, call.args().size()); + } + @Test public void testCallWithPositionalArgs() throws ParseException { CallStatement call = - (CallStatement) parser.parsePlan("CALL c.n.func(1, '2', 3L, true, 1.0D, 9.0e1, 900e-1BD)"); + (CallStatement) + parser.parsePlan( + "CALL c.system.rollback_to_snapshot(1, '2', 3L, true, 1.0D, 9.0e1, 900e-1BD)"); Assert.assertEquals( - ImmutableList.of("c", "n", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("c", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(7, call.args().size()); @@ -94,9 +119,12 @@ public void testCallWithPositionalArgs() throws ParseException { @Test public void testCallWithNamedArgs() throws ParseException { CallStatement call = - (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, c2 => '2', c3 => true)"); + (CallStatement) + parser.parsePlan( + "CALL cat.system.rollback_to_snapshot(c1 => 1, c2 => '2', c3 => true)"); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(3, call.args().size()); @@ -107,9 +135,11 @@ public void testCallWithNamedArgs() throws ParseException { @Test public void testCallWithMixedArgs() throws ParseException { - CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, '2')"); + CallStatement call = + (CallStatement) parser.parsePlan("CALL cat.system.rollback_to_snapshot(c1 => 1, '2')"); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(2, call.args().size()); @@ -121,9 +151,11 @@ public void testCallWithMixedArgs() throws ParseException { public void testCallWithTimestampArg() throws ParseException { CallStatement call = (CallStatement) - parser.parsePlan("CALL cat.system.func(TIMESTAMP '2017-02-03T10:37:30.00Z')"); + parser.parsePlan( + "CALL cat.system.rollback_to_snapshot(TIMESTAMP '2017-02-03T10:37:30.00Z')"); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(1, call.args().size()); @@ -134,9 +166,11 @@ public void testCallWithTimestampArg() throws ParseException { @Test public void testCallWithVarSubstitution() throws ParseException { CallStatement call = - (CallStatement) parser.parsePlan("CALL cat.system.func('${spark.extra.prop}')"); + (CallStatement) + parser.parsePlan("CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')"); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(1, call.args().size()); @@ -145,30 +179,32 @@ public void testCallWithVarSubstitution() throws ParseException { @Test public void testCallParseError() { - assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) + assertThatThrownBy(() -> parser.parsePlan("CALL cat.system.rollback_to_snapshot kebab")) .as("Should fail with a sensible parse error") .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("missing '(' at 'radish'"); + .hasMessageContaining("missing '(' at 'kebab'"); } @Test public void testCallStripsComments() throws ParseException { List callStatementsWithComments = Lists.newArrayList( - "/* bracketed comment */ CALL cat.system.func('${spark.extra.prop}')", - "/**/ CALL cat.system.func('${spark.extra.prop}')", - "-- single line comment \n CALL cat.system.func('${spark.extra.prop}')", - "-- multiple \n-- single line \n-- comments \n CALL cat.system.func('${spark.extra.prop}')", - "/* select * from multiline_comment \n where x like '%sql%'; */ CALL cat.system.func('${spark.extra.prop}')", + "/* bracketed comment */ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "/**/ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "-- single line comment \n CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "-- multiple \n-- single line \n-- comments \n CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", + "/* select * from multiline_comment \n where x like '%sql%'; */ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", "/* {\"app\": \"dbt\", \"dbt_version\": \"1.0.1\", \"profile_name\": \"profile1\", \"target_name\": \"dev\", " - + "\"node_id\": \"model.profile1.stg_users\"} \n*/ CALL cat.system.func('${spark.extra.prop}')", + + "\"node_id\": \"model.profile1.stg_users\"} \n*/ CALL cat.system.rollback_to_snapshot('${spark.extra.prop}')", "/* Some multi-line comment \n" - + "*/ CALL /* inline comment */ cat.system.func('${spark.extra.prop}') -- ending comment", - "CALL -- a line ending comment\n" + "cat.system.func('${spark.extra.prop}')"); + + "*/ CALL /* inline comment */ cat.system.rollback_to_snapshot('${spark.extra.prop}') -- ending comment", + "CALL -- a line ending comment\n" + + "cat.system.rollback_to_snapshot('${spark.extra.prop}')"); for (String sqlText : callStatementsWithComments) { CallStatement call = (CallStatement) parser.parsePlan(sqlText); Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + ImmutableList.of("cat", "system", "rollback_to_snapshot"), + JavaConverters.seqAsJavaList(call.name())); Assert.assertEquals(1, call.args().size()); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java index b489cf76f1ef..8ac21fa0c7a3 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java @@ -31,8 +31,9 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Test; public class TestCherrypickSnapshotProcedure extends SparkExtensionsTestBase { @@ -178,8 +179,13 @@ public void testInvalidCherrypickSnapshotCases() { assertThatThrownBy(() -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") - .isInstanceOf(NoSuchProcedureException.class) - .hasMessageContaining("not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) .as("Should reject calls without all required args") diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index e9f42f5e0195..25dc7e47e971 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -52,7 +52,7 @@ import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -178,8 +178,12 @@ public void testInvalidExpireSnapshotsCases() { assertThatThrownBy(() -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") - .isInstanceOf(NoSuchProcedureException.class) - .hasMessageContaining("not found"); + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) .as("Should reject calls without all required args") diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java index 37d40957bc44..c960b0b98660 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java @@ -30,8 +30,9 @@ import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Test; public class TestFastForwardBranchProcedure extends SparkExtensionsTestBase { @@ -176,8 +177,13 @@ public void testInvalidFastForwardBranchCases() { assertThatThrownBy( () -> sql("CALL %s.custom.fast_forward('test_table', 'main', 'newBranch')", catalogName)) - .isInstanceOf(NoSuchProcedureException.class) - .hasMessage("Procedure custom.fast_forward not found"); + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.fast_forward('test_table', 'main')", catalogName)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java index e71ad687293b..5c65d7184f07 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java @@ -31,8 +31,9 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Test; public class TestPublishChangesProcedure extends SparkExtensionsTestBase { @@ -176,8 +177,12 @@ public void testInvalidApplyWapChangesCases() { assertThatThrownBy( () -> sql("CALL %s.custom.publish_changes('n', 't', 'not_valid')", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") - .isInstanceOf(NoSuchProcedureException.class) - .hasMessageContaining("not found"); + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) .as("Should reject calls without all required args") diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 6e193b9a8642..e1966520893a 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -63,7 +63,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; @@ -266,8 +265,12 @@ public void testInvalidRemoveOrphanFilesCases() { assertThatThrownBy(() -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") - .isInstanceOf(NoSuchProcedureException.class) - .hasMessageContaining("not found"); + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) .as("Should reject calls without all required args") diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 1418d82cebf8..fd8754b30d5b 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -40,7 +40,7 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -566,8 +566,12 @@ public void testInvalidCasesForRewriteDataFiles() { assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") - .isInstanceOf(NoSuchProcedureException.class) - .hasMessageContaining("not found"); + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) .as("Should reject calls without all required args") diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 591d88e83fca..dd4c55078a9d 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -30,8 +30,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -284,8 +284,12 @@ public void testInvalidRewriteManifestsCases() { assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") - .isInstanceOf(NoSuchProcedureException.class) - .hasMessageContaining("not found"); + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) .as("Should reject calls without all required args") diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java index b60a769f9f26..f8e20714017c 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java @@ -29,8 +29,9 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -261,8 +262,12 @@ public void testInvalidRollbackToSnapshotCases() { assertThatThrownBy(() -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") - .isInstanceOf(NoSuchProcedureException.class) - .hasMessageContaining("not found"); + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) .as("Should reject calls without all required args") diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java index c65be60cc1a7..48cf9648fab9 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java @@ -31,8 +31,9 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -303,8 +304,12 @@ public void testInvalidRollbackToTimestampCases() { assertThatThrownBy( () -> sql("CALL %s.custom.rollback_to_timestamp('n', 't', %s)", catalogName, timestamp)) .as("Should not resolve procedures in arbitrary namespaces") - .isInstanceOf(NoSuchProcedureException.class) - .hasMessageContaining("not found"); + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) .as("Should reject calls without all required args") diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java index 55bda0df1b3c..a03a44f4f97d 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java @@ -30,8 +30,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.spark.sql.AnalysisException; -import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.apache.spark.sql.catalyst.parser.ParseException; import org.junit.After; +import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -213,8 +214,12 @@ public void testInvalidRollbackToSnapshotCases() { assertThatThrownBy(() -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") - .isInstanceOf(NoSuchProcedureException.class) - .hasMessageContaining("not found"); + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + Assert.assertEquals("PARSE_SYNTAX_ERROR", parseException.getErrorClass()); + Assert.assertEquals("Syntax error at or near 'CALL'", parseException.message()); + }); assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) .as("Should reject calls without all required args") diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java index b324cd4422b3..42003b24e94c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java @@ -20,6 +20,7 @@ import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.function.Supplier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -37,6 +38,10 @@ public static ProcedureBuilder newBuilder(String name) { return builderSupplier != null ? builderSupplier.get() : null; } + public static Set names() { + return BUILDERS.keySet(); + } + private static Map> initProcedureBuilders() { ImmutableMap.Builder> mapBuilder = ImmutableMap.builder(); mapBuilder.put("rollback_to_snapshot", RollbackToSnapshotProcedure::builder); From e71e3cb1954fec66dc8915da2854e8fe02d08197 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Tue, 19 Nov 2024 14:23:55 +0100 Subject: [PATCH 159/313] Core: Inherited classes from SnapshotProducer has TableOperations redundantly as member (#11578) --- .../apache/iceberg/BaseRewriteManifests.java | 15 +++--- .../java/org/apache/iceberg/FastAppend.java | 15 +++--- .../iceberg/MergingSnapshotProducer.java | 47 +++++++++---------- .../org/apache/iceberg/SnapshotProducer.java | 4 ++ 4 files changed, 41 insertions(+), 40 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index dce6d4a995bd..ef578a82f3fa 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -52,7 +52,6 @@ public class BaseRewriteManifests extends SnapshotProducer private static final String REPLACED_MANIFESTS_COUNT = "manifests-replaced"; private static final String PROCESSED_ENTRY_COUNT = "entries-processed"; - private final TableOperations ops; private final Map specsById; private final long manifestTargetSizeBytes; @@ -74,10 +73,10 @@ public class BaseRewriteManifests extends SnapshotProducer BaseRewriteManifests(TableOperations ops) { super(ops); - this.ops = ops; - this.specsById = ops.current().specsById(); + this.specsById = ops().current().specsById(); this.manifestTargetSizeBytes = - ops.current() + ops() + .current() .propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT); } @@ -153,8 +152,8 @@ public RewriteManifests addManifest(ManifestFile manifest) { } private ManifestFile copyManifest(ManifestFile manifest) { - TableMetadata current = ops.current(); - InputFile toCopy = ops.io().newInputFile(manifest); + TableMetadata current = ops().current(); + InputFile toCopy = ops().io().newInputFile(manifest); EncryptedOutputFile newFile = newManifestOutputFile(); return ManifestFiles.copyRewriteManifest( current.formatVersion(), @@ -168,7 +167,7 @@ private ManifestFile copyManifest(ManifestFile manifest) { @Override public List apply(TableMetadata base, Snapshot snapshot) { - List currentManifests = base.currentSnapshot().allManifests(ops.io()); + List currentManifests = base.currentSnapshot().allManifests(ops().io()); Set currentManifestSet = ImmutableSet.copyOf(currentManifests); validateDeletedManifests(currentManifestSet, base.currentSnapshot().snapshotId()); @@ -246,7 +245,7 @@ private void performRewrite(List currentManifests) { } else { rewrittenManifests.add(manifest); try (ManifestReader reader = - ManifestFiles.read(manifest, ops.io(), ops.current().specsById()) + ManifestFiles.read(manifest, ops().io(), ops().current().specsById()) .select(Collections.singletonList("*"))) { reader .liveEntries() diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 1e2f6fe0d90d..51c0d5926fdb 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -34,7 +34,6 @@ /** {@link AppendFiles Append} implementation that adds a new manifest file for the write. */ class FastAppend extends SnapshotProducer implements AppendFiles { private final String tableName; - private final TableOperations ops; private final PartitionSpec spec; private final SnapshotSummary.Builder summaryBuilder = SnapshotSummary.builder(); private final DataFileSet newFiles = DataFileSet.create(); @@ -46,8 +45,7 @@ class FastAppend extends SnapshotProducer implements AppendFiles { FastAppend(String tableName, TableOperations ops) { super(ops); this.tableName = tableName; - this.ops = ops; - this.spec = ops.current().spec(); + this.spec = ops().current().spec(); } @Override @@ -69,7 +67,8 @@ protected String operation() { @Override protected Map summary() { summaryBuilder.setPartitionSummaryLimit( - ops.current() + ops() + .current() .propertyAsInt( TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, TableProperties.WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT)); @@ -118,8 +117,8 @@ public FastAppend appendManifest(ManifestFile manifest) { } private ManifestFile copyManifest(ManifestFile manifest) { - TableMetadata current = ops.current(); - InputFile toCopy = ops.io().newInputFile(manifest); + TableMetadata current = ops().current(); + InputFile toCopy = ops().io().newInputFile(manifest); EncryptedOutputFile newManifestFile = newManifestOutputFile(); return ManifestFiles.copyAppendManifest( current.formatVersion(), @@ -151,7 +150,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { Iterables.addAll(manifests, appendManifestsWithMetadata); if (snapshot != null) { - manifests.addAll(snapshot.allManifests(ops.io())); + manifests.addAll(snapshot.allManifests(ops().io())); } return manifests; @@ -160,7 +159,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { @Override public Object updateEvent() { long snapshotId = snapshotId(); - Snapshot snapshot = ops.current().snapshot(snapshotId); + Snapshot snapshot = ops().current().snapshot(snapshotId); long sequenceNumber = snapshot.sequenceNumber(); return new CreateSnapshotEvent( tableName, operation(), snapshotId, sequenceNumber, snapshot.summary()); diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 41f0ad00178c..75dd7410115e 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -77,7 +77,6 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { ImmutableSet.of(DataOperations.OVERWRITE, DataOperations.DELETE, DataOperations.REPLACE); private final String tableName; - private final TableOperations ops; private final SnapshotSummary.Builder summaryBuilder = SnapshotSummary.builder(); private final ManifestMergeManager mergeManager; private final ManifestFilterManager filterManager; @@ -108,7 +107,6 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { MergingSnapshotProducer(String tableName, TableOperations ops) { super(ops); this.tableName = tableName; - this.ops = ops; long targetSizeBytes = ops.current() .propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT); @@ -246,7 +244,7 @@ protected void add(DataFile file) { } private PartitionSpec spec(int specId) { - return ops.current().spec(specId); + return ops().current().spec(specId); } /** Add a delete file to the new snapshot. */ @@ -304,7 +302,7 @@ protected void validateNewDeleteFile(DeleteFile file) { } private int formatVersion() { - return ops.current().formatVersion(); + return ops().current().formatVersion(); } /** Add all files in a manifest to the new snapshot. */ @@ -322,8 +320,8 @@ protected void add(ManifestFile manifest) { } private ManifestFile copyManifest(ManifestFile manifest) { - TableMetadata current = ops.current(); - InputFile toCopy = ops.io().newInputFile(manifest); + TableMetadata current = ops().current(); + InputFile toCopy = ops().io().newInputFile(manifest); EncryptedOutputFile newManifestFile = newManifestOutputFile(); return ManifestFiles.copyAppendManifest( current.formatVersion(), @@ -427,7 +425,7 @@ private CloseableIterable> addedDataFiles( Set newSnapshots = history.second(); ManifestGroup manifestGroup = - new ManifestGroup(ops.io(), manifests, ImmutableList.of()) + new ManifestGroup(ops().io(), manifests, ImmutableList.of()) .caseSensitive(caseSensitive) .filterManifestEntries(entry -> newSnapshots.contains(entry.snapshotId())) .specsById(base.specsById()) @@ -590,7 +588,7 @@ protected DeleteFileIndex addedDeleteFiles( Snapshot parent) { // if there is no current table state, return empty delete file index if (parent == null || base.formatVersion() < 2) { - return DeleteFileIndex.builderFor(ops.io(), ImmutableList.of()) + return DeleteFileIndex.builderFor(ops().io(), ImmutableList.of()) .specsById(base.specsById()) .build(); } @@ -698,7 +696,7 @@ private CloseableIterable> deletedDataFiles( Set newSnapshots = history.second(); ManifestGroup manifestGroup = - new ManifestGroup(ops.io(), manifests, ImmutableList.of()) + new ManifestGroup(ops().io(), manifests, ImmutableList.of()) .caseSensitive(caseSensitive) .filterManifestEntries(entry -> newSnapshots.contains(entry.snapshotId())) .filterManifestEntries(entry -> entry.status().equals(ManifestEntry.Status.DELETED)) @@ -737,10 +735,10 @@ private DeleteFileIndex buildDeleteFileIndex( Expression dataFilter, PartitionSet partitionSet) { DeleteFileIndex.Builder builder = - DeleteFileIndex.builderFor(ops.io(), deleteManifests) + DeleteFileIndex.builderFor(ops().io(), deleteManifests) .afterSequenceNumber(startingSequenceNumber) .caseSensitive(caseSensitive) - .specsById(ops.current().specsById()); + .specsById(ops().current().specsById()); if (dataFilter != null) { builder.filterData(dataFilter); @@ -778,7 +776,7 @@ protected void validateDataFilesExist( Set newSnapshots = history.second(); ManifestGroup matchingDeletesGroup = - new ManifestGroup(ops.io(), manifests, ImmutableList.of()) + new ManifestGroup(ops().io(), manifests, ImmutableList.of()) .filterManifestEntries( entry -> entry.status() != ManifestEntry.Status.ADDED @@ -836,7 +834,7 @@ protected void validateAddedDVs( private void validateAddedDVs( ManifestFile manifest, Expression conflictDetectionFilter, Set newSnapshotIds) { try (CloseableIterable> entries = - ManifestFiles.readDeleteManifest(manifest, ops.io(), ops.current().specsById()) + ManifestFiles.readDeleteManifest(manifest, ops().io(), ops().current().specsById()) .filterRows(conflictDetectionFilter) .caseSensitive(caseSensitive) .liveEntries()) { @@ -875,13 +873,13 @@ private Pair, Set> validationHistory( if (matchingOperations.contains(currentSnapshot.operation())) { newSnapshots.add(currentSnapshot.snapshotId()); if (content == ManifestContent.DATA) { - for (ManifestFile manifest : currentSnapshot.dataManifests(ops.io())) { + for (ManifestFile manifest : currentSnapshot.dataManifests(ops().io())) { if (manifest.snapshotId() == currentSnapshot.snapshotId()) { manifests.add(manifest); } } } else { - for (ManifestFile manifest : currentSnapshot.deleteManifests(ops.io())) { + for (ManifestFile manifest : currentSnapshot.deleteManifests(ops().io())) { if (manifest.snapshotId() == currentSnapshot.snapshotId()) { manifests.add(manifest); } @@ -902,7 +900,8 @@ private Pair, Set> validationHistory( @Override protected Map summary() { summaryBuilder.setPartitionSummaryLimit( - ops.current() + ops() + .current() .propertyAsInt( TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, TableProperties.WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT)); @@ -915,7 +914,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { List filtered = filterManager.filterManifests( SnapshotUtil.schemaFor(base, targetBranch()), - snapshot != null ? snapshot.dataManifests(ops.io()) : null); + snapshot != null ? snapshot.dataManifests(ops().io()) : null); long minDataSequenceNumber = filtered.stream() .map(ManifestFile::minSequenceNumber) @@ -929,7 +928,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { List filteredDeletes = deleteFilterManager.filterManifests( SnapshotUtil.schemaFor(base, targetBranch()), - snapshot != null ? snapshot.deleteManifests(ops.io()) : null); + snapshot != null ? snapshot.deleteManifests(ops().io()) : null); // only keep manifests that have live data files or that were written by this commit Predicate shouldKeep = @@ -959,7 +958,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { @Override public Object updateEvent() { long snapshotId = snapshotId(); - Snapshot justSaved = ops.refresh().snapshot(snapshotId); + Snapshot justSaved = ops().refresh().snapshot(snapshotId); long sequenceNumber = TableMetadata.INVALID_SEQUENCE_NUMBER; Map summary; if (justSaved == null) { @@ -1086,7 +1085,7 @@ private List newDeleteFilesAsManifests() { if (cachedNewDeleteManifests.isEmpty()) { newDeleteFilesBySpec.forEach( (specId, deleteFiles) -> { - PartitionSpec spec = ops.current().spec(specId); + PartitionSpec spec = ops().current().spec(specId); List newDeleteManifests = writeDeleteManifests(deleteFiles, spec); cachedNewDeleteManifests.addAll(newDeleteManifests); }); @@ -1099,7 +1098,7 @@ private List newDeleteFilesAsManifests() { private class DataFileFilterManager extends ManifestFilterManager { private DataFileFilterManager() { - super(ops.current().specsById(), MergingSnapshotProducer.this::workerPool); + super(ops().current().specsById(), MergingSnapshotProducer.this::workerPool); } @Override @@ -1136,7 +1135,7 @@ protected long snapshotId() { @Override protected PartitionSpec spec(int specId) { - return ops.current().spec(specId); + return ops().current().spec(specId); } @Override @@ -1157,7 +1156,7 @@ protected ManifestReader newManifestReader(ManifestFile manifest) { private class DeleteFileFilterManager extends ManifestFilterManager { private DeleteFileFilterManager() { - super(ops.current().specsById(), MergingSnapshotProducer.this::workerPool); + super(ops().current().specsById(), MergingSnapshotProducer.this::workerPool); } @Override @@ -1194,7 +1193,7 @@ protected long snapshotId() { @Override protected PartitionSpec spec(int specId) { - return ops.current().spec(specId); + return ops().current().spec(specId); } @Override diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 45b71d654344..f5a7e99b684e 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -159,6 +159,10 @@ public ThisT scanManifestsWith(ExecutorService executorService) { return self(); } + protected TableOperations ops() { + return ops; + } + protected CommitMetrics commitMetrics() { if (commitMetrics == null) { this.commitMetrics = CommitMetrics.of(new DefaultMetricsContext()); From 3badfe0c1fcf0c0adfc7aa4a10f0b50365c48cf9 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 19 Nov 2024 15:38:46 +0100 Subject: [PATCH 160/313] =?UTF-8?q?Revert=20"Core:=20Use=20encoding/decodi?= =?UTF-8?q?ng=20methods=20for=20namespaces=20and=20deprecate=20Spli?= =?UTF-8?q?=E2=80=A6"=20(#11574)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit 5fc1413a5efc4419ccc081f3031325f107ccddab. --- .../apache/iceberg/rest/RESTSessionCatalog.java | 2 +- .../java/org/apache/iceberg/rest/RESTUtil.java | 16 +++------------- .../apache/iceberg/rest/RESTCatalogAdapter.java | 6 +++++- 3 files changed, 9 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index cc42604f700d..b89595635314 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -615,7 +615,7 @@ public List listNamespaces(SessionContext context, Namespace namespac Map queryParams = Maps.newHashMap(); if (!namespace.isEmpty()) { - queryParams.put("parent", RESTUtil.encodeNamespace(namespace)); + queryParams.put("parent", RESTUtil.NAMESPACE_JOINER.join(namespace.levels())); } ImmutableList.Builder namespaces = ImmutableList.builder(); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java b/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java index 45422b8ae8b5..fab01162cad7 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java @@ -33,24 +33,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; public class RESTUtil { + private static final char NAMESPACE_SEPARATOR = '\u001f'; + public static final Joiner NAMESPACE_JOINER = Joiner.on(NAMESPACE_SEPARATOR); + public static final Splitter NAMESPACE_SPLITTER = Splitter.on(NAMESPACE_SEPARATOR); private static final String NAMESPACE_ESCAPED_SEPARATOR = "%1F"; private static final Joiner NAMESPACE_ESCAPED_JOINER = Joiner.on(NAMESPACE_ESCAPED_SEPARATOR); private static final Splitter NAMESPACE_ESCAPED_SPLITTER = Splitter.on(NAMESPACE_ESCAPED_SEPARATOR); - /** - * @deprecated since 1.7.0, will be made private in 1.8.0; use {@link - * RESTUtil#encodeNamespace(Namespace)} instead. - */ - @Deprecated public static final Joiner NAMESPACE_JOINER = Joiner.on(NAMESPACE_ESCAPED_SEPARATOR); - - /** - * @deprecated since 1.7.0, will be made private in 1.8.0; use {@link - * RESTUtil#decodeNamespace(String)} instead. - */ - @Deprecated - public static final Splitter NAMESPACE_SPLITTER = Splitter.on(NAMESPACE_ESCAPED_SEPARATOR); - private RESTUtil() {} public static String stripTrailingSlash(String path) { diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 6477dfcd00eb..aa77b5ad10b6 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -298,7 +298,11 @@ public T handleRequest( if (asNamespaceCatalog != null) { Namespace ns; if (vars.containsKey("parent")) { - ns = RESTUtil.decodeNamespace(vars.get("parent")); + ns = + Namespace.of( + RESTUtil.NAMESPACE_SPLITTER + .splitToStream(vars.get("parent")) + .toArray(String[]::new)); } else { ns = Namespace.empty(); } From f6d02de77fd8932cbe1fdf97e0f00acc3368e656 Mon Sep 17 00:00:00 2001 From: leesf Date: Wed, 20 Nov 2024 14:05:06 +0800 Subject: [PATCH 161/313] Core: Delete temp metadata file when version already exists (#11350) --- .../org/apache/iceberg/hadoop/HadoopTableOperations.java | 9 ++++++++- .../org/apache/iceberg/hadoop/TestHadoopCommits.java | 7 +++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java index 24299371401c..1f3c9182bf5f 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java @@ -365,7 +365,14 @@ private void renameToFinal(FileSystem fs, Path src, Path dst, int nextVersion) { } if (fs.exists(dst)) { - throw new CommitFailedException("Version %d already exists: %s", nextVersion, dst); + CommitFailedException cfe = + new CommitFailedException("Version %d already exists: %s", nextVersion, dst); + RuntimeException re = tryDelete(src); + if (re != null) { + cfe.addSuppressed(re); + } + + throw cfe; } if (!fs.rename(src, dst)) { diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java index 60bef7fe0b14..a8139180ca7d 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java @@ -205,6 +205,13 @@ public void testFailedCommit() throws Exception { List manifests = listManifestFiles(); assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); + + // verifies that there is no temporary metadata.json files left on disk + List actual = + listMetadataJsonFiles().stream().map(File::getName).sorted().collect(Collectors.toList()); + assertThat(actual) + .as("only v1 and v2 metadata.json should exist.") + .containsExactly("v1.metadata.json", "v2.metadata.json"); } @Test From 657fa86b4928de23bf01ceda0f6f6112bc19403c Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 20 Nov 2024 09:35:34 +0100 Subject: [PATCH 162/313] Build: Bump Apache Parquet 1.14.4 (#11502) * Revert "Revert "Build: Bump parquet from 1.13.1 to 1.14.3 (#11264)" (#11462)" This reverts commit 7cc16fa94d7cd4e19397e9b4fba62185e0fa5eac. * Bump to Parquet 1.14.4 * Lookup sizes instead * Update build.gradle --- .../TestMetadataTableReadableMetrics.java | 97 ++++++++++++++++--- .../TestMetadataTableReadableMetrics.java | 95 +++++++++++++++--- .../TestMetadataTableReadableMetrics.java | 95 +++++++++++++++--- gradle/libs.versions.toml | 2 +- 4 files changed, 242 insertions(+), 47 deletions(-) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 40dfda723749..488969bab045 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -28,6 +28,7 @@ import java.nio.file.Path; import java.util.Base64; import java.util.List; +import java.util.Map; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.table.api.TableEnvironment; @@ -137,7 +138,7 @@ private Table createPrimitiveTable() throws IOException { return table; } - private void createNestedTable() throws IOException { + private Table createNestedTable() throws IOException { Table table = validationCatalog.createTable( TableIdentifier.of(DATABASE, TABLE_NAME), @@ -154,6 +155,8 @@ private void createNestedTable() throws IOException { File testFile = File.createTempFile("junit", null, temp.toFile()); DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); table.newAppend().appendFile(dataFile).commit(); + + return table; } @BeforeEach @@ -168,7 +171,7 @@ public void before() { @AfterEach public void clean() { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); super.clean(); } @@ -212,32 +215,88 @@ protected Object[] row(Object... values) { @TestTemplate public void testPrimitiveColumns() throws Exception { - createPrimitiveTable(); + Table table = createPrimitiveTable(); List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); + // With new releases of Parquet, new features might be added which cause the + // size of the column to increase. For example, with Parquet 1.14.x the + // uncompressed size has been added to allow for better allocation of memory upfront. + // Therefore, we look the sizes up, rather than hardcoding them + DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + Map columnSizeStats = dataFile.columnSizes(); + Row binaryCol = Row.of( - 52L, + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("binaryCol").fieldId()), 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("booleanCol").fieldId()), + 4L, + 0L, + null, + false, + true); + Row decimalCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("decimalCol").fieldId()), + 4L, + 1L, + null, + new BigDecimal("1.00"), + new BigDecimal("2.00")); + Row doubleCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("doubleCol").fieldId()), + 4L, + 0L, + 1L, + 1.0D, + 2.0D); Row fixedCol = Row.of( - 44L, + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("fixedCol").fieldId()), 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); + Row floatCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("floatCol").fieldId()), + 4L, + 0L, + 2L, + 0f, + 0f); + Row intCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("intCol").fieldId()), + 4L, + 0L, + null, + 1, + 2); + Row longCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("longCol").fieldId()), + 4L, + 0L, + null, + 1L, + 2L); + Row stringCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("stringCol").fieldId()), + 4L, + 0L, + null, + "1", + "2"); List expected = Lists.newArrayList( @@ -288,12 +347,18 @@ public void testSelectNestedValues() throws Exception { @TestTemplate public void testNestedValues() throws Exception { createNestedTable(); + List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); + + // We have to take a slightly different approach, since we don't store + // the column sizes for nested fields. + long leafDoubleColSize = + (long) ((Row) ((Row) result.get(0).getField(0)).getField(0)).getField(0); + long leafLongColSize = (long) ((Row) ((Row) result.get(0).getField(0)).getField(1)).getField(0); - Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(leafDoubleColSize, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(leafLongColSize, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); - TestHelpers.assertRows( - sql("SELECT readable_metrics FROM %s$files", TABLE_NAME), ImmutableList.of(metrics)); + TestHelpers.assertRows(result, ImmutableList.of(metrics)); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 9cf953342a18..488969bab045 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -28,6 +28,7 @@ import java.nio.file.Path; import java.util.Base64; import java.util.List; +import java.util.Map; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.table.api.TableEnvironment; @@ -137,7 +138,7 @@ private Table createPrimitiveTable() throws IOException { return table; } - private void createNestedTable() throws IOException { + private Table createNestedTable() throws IOException { Table table = validationCatalog.createTable( TableIdentifier.of(DATABASE, TABLE_NAME), @@ -154,6 +155,8 @@ private void createNestedTable() throws IOException { File testFile = File.createTempFile("junit", null, temp.toFile()); DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); table.newAppend().appendFile(dataFile).commit(); + + return table; } @BeforeEach @@ -212,32 +215,88 @@ protected Object[] row(Object... values) { @TestTemplate public void testPrimitiveColumns() throws Exception { - createPrimitiveTable(); + Table table = createPrimitiveTable(); List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); + // With new releases of Parquet, new features might be added which cause the + // size of the column to increase. For example, with Parquet 1.14.x the + // uncompressed size has been added to allow for better allocation of memory upfront. + // Therefore, we look the sizes up, rather than hardcoding them + DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + Map columnSizeStats = dataFile.columnSizes(); + Row binaryCol = Row.of( - 52L, + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("binaryCol").fieldId()), 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("booleanCol").fieldId()), + 4L, + 0L, + null, + false, + true); + Row decimalCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("decimalCol").fieldId()), + 4L, + 1L, + null, + new BigDecimal("1.00"), + new BigDecimal("2.00")); + Row doubleCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("doubleCol").fieldId()), + 4L, + 0L, + 1L, + 1.0D, + 2.0D); Row fixedCol = Row.of( - 44L, + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("fixedCol").fieldId()), 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); + Row floatCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("floatCol").fieldId()), + 4L, + 0L, + 2L, + 0f, + 0f); + Row intCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("intCol").fieldId()), + 4L, + 0L, + null, + 1, + 2); + Row longCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("longCol").fieldId()), + 4L, + 0L, + null, + 1L, + 2L); + Row stringCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("stringCol").fieldId()), + 4L, + 0L, + null, + "1", + "2"); List expected = Lists.newArrayList( @@ -288,12 +347,18 @@ public void testSelectNestedValues() throws Exception { @TestTemplate public void testNestedValues() throws Exception { createNestedTable(); + List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); + + // We have to take a slightly different approach, since we don't store + // the column sizes for nested fields. + long leafDoubleColSize = + (long) ((Row) ((Row) result.get(0).getField(0)).getField(0)).getField(0); + long leafLongColSize = (long) ((Row) ((Row) result.get(0).getField(0)).getField(1)).getField(0); - Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(leafDoubleColSize, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(leafLongColSize, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); - TestHelpers.assertRows( - sql("SELECT readable_metrics FROM %s$files", TABLE_NAME), ImmutableList.of(metrics)); + TestHelpers.assertRows(result, ImmutableList.of(metrics)); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 9cf953342a18..488969bab045 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -28,6 +28,7 @@ import java.nio.file.Path; import java.util.Base64; import java.util.List; +import java.util.Map; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.table.api.TableEnvironment; @@ -137,7 +138,7 @@ private Table createPrimitiveTable() throws IOException { return table; } - private void createNestedTable() throws IOException { + private Table createNestedTable() throws IOException { Table table = validationCatalog.createTable( TableIdentifier.of(DATABASE, TABLE_NAME), @@ -154,6 +155,8 @@ private void createNestedTable() throws IOException { File testFile = File.createTempFile("junit", null, temp.toFile()); DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); table.newAppend().appendFile(dataFile).commit(); + + return table; } @BeforeEach @@ -212,32 +215,88 @@ protected Object[] row(Object... values) { @TestTemplate public void testPrimitiveColumns() throws Exception { - createPrimitiveTable(); + Table table = createPrimitiveTable(); List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); + // With new releases of Parquet, new features might be added which cause the + // size of the column to increase. For example, with Parquet 1.14.x the + // uncompressed size has been added to allow for better allocation of memory upfront. + // Therefore, we look the sizes up, rather than hardcoding them + DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + Map columnSizeStats = dataFile.columnSizes(); + Row binaryCol = Row.of( - 52L, + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("binaryCol").fieldId()), 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row booleanCol = Row.of(32L, 4L, 0L, null, false, true); - Row decimalCol = Row.of(85L, 4L, 1L, null, new BigDecimal("1.00"), new BigDecimal("2.00")); - Row doubleCol = Row.of(85L, 4L, 0L, 1L, 1.0D, 2.0D); + Row booleanCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("booleanCol").fieldId()), + 4L, + 0L, + null, + false, + true); + Row decimalCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("decimalCol").fieldId()), + 4L, + 1L, + null, + new BigDecimal("1.00"), + new BigDecimal("2.00")); + Row doubleCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("doubleCol").fieldId()), + 4L, + 0L, + 1L, + 1.0D, + 2.0D); Row fixedCol = Row.of( - 44L, + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("fixedCol").fieldId()), 4L, 2L, null, Base64.getDecoder().decode("1111"), Base64.getDecoder().decode("2222")); - Row floatCol = Row.of(71L, 4L, 0L, 2L, 0f, 0f); - Row intCol = Row.of(71L, 4L, 0L, null, 1, 2); - Row longCol = Row.of(79L, 4L, 0L, null, 1L, 2L); - Row stringCol = Row.of(79L, 4L, 0L, null, "1", "2"); + Row floatCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("floatCol").fieldId()), + 4L, + 0L, + 2L, + 0f, + 0f); + Row intCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("intCol").fieldId()), + 4L, + 0L, + null, + 1, + 2); + Row longCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("longCol").fieldId()), + 4L, + 0L, + null, + 1L, + 2L); + Row stringCol = + Row.of( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("stringCol").fieldId()), + 4L, + 0L, + null, + "1", + "2"); List expected = Lists.newArrayList( @@ -288,12 +347,18 @@ public void testSelectNestedValues() throws Exception { @TestTemplate public void testNestedValues() throws Exception { createNestedTable(); + List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); + + // We have to take a slightly different approach, since we don't store + // the column sizes for nested fields. + long leafDoubleColSize = + (long) ((Row) ((Row) result.get(0).getField(0)).getField(0)).getField(0); + long leafLongColSize = (long) ((Row) ((Row) result.get(0).getField(0)).getField(1)).getField(0); - Row leafDoubleCol = Row.of(46L, 3L, 1L, 1L, 0.0D, 0.0D); - Row leafLongCol = Row.of(54L, 3L, 1L, null, 0L, 1L); + Row leafDoubleCol = Row.of(leafDoubleColSize, 3L, 1L, 1L, 0.0D, 0.0D); + Row leafLongCol = Row.of(leafLongColSize, 3L, 1L, null, 0L, 1L); Row metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); - TestHelpers.assertRows( - sql("SELECT readable_metrics FROM %s$files", TABLE_NAME), ImmutableList.of(metrics)); + TestHelpers.assertRows(result, ImmutableList.of(metrics)); } } diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 8af0d6ec6ab2..7358d0b8547a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -75,7 +75,7 @@ netty-buffer = "4.1.115.Final" netty-buffer-compat = "4.1.115.Final" object-client-bundle = "3.3.2" orc = "1.9.5" -parquet = "1.13.1" +parquet = "1.14.4" roaringbitmap = "1.3.0" scala-collection-compat = "2.12.0" slf4j = "2.0.16" From 3c4a710d2f087ff2d1220607424e9164a6f09281 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 20 Nov 2024 09:48:27 +0100 Subject: [PATCH 163/313] Core: Filter on live entries when reading the manifest (#9996) --- .../apache/iceberg/ManifestFilterManager.java | 54 +++++++++---------- 1 file changed, 26 insertions(+), 28 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index 564e27a937a1..518599b99c6b 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -431,7 +431,7 @@ private ManifestFile filterManifestWithDeletedFiles( ManifestWriter writer = newManifestWriter(reader.spec()); try { reader - .entries() + .liveEntries() .forEach( entry -> { F file = entry.file(); @@ -443,38 +443,36 @@ private ManifestFile filterManifestWithDeletedFiles( && entry.isLive() && entry.dataSequenceNumber() > 0 && entry.dataSequenceNumber() < minSequenceNumber); - if (entry.status() != ManifestEntry.Status.DELETED) { - if (markedForDelete || evaluator.rowsMightMatch(file)) { - boolean allRowsMatch = markedForDelete || evaluator.rowsMustMatch(file); - ValidationException.check( - allRowsMatch - || isDelete, // ignore delete files where some records may not match - // the expression - "Cannot delete file where some, but not all, rows match filter %s: %s", - this.deleteExpression, - file.location()); - - if (allRowsMatch) { - writer.delete(entry); - - if (deletedFiles.contains(file)) { - LOG.warn( - "Deleting a duplicate path from manifest {}: {}", - manifest.path(), - file.location()); - duplicateDeleteCount += 1; - } else { - // only add the file to deletes if it is a new delete - // this keeps the snapshot summary accurate for non-duplicate data - deletedFiles.add(file.copyWithoutStats()); - } + if (markedForDelete || evaluator.rowsMightMatch(file)) { + boolean allRowsMatch = markedForDelete || evaluator.rowsMustMatch(file); + ValidationException.check( + allRowsMatch + || isDelete, // ignore delete files where some records may not match + // the expression + "Cannot delete file where some, but not all, rows match filter %s: %s", + this.deleteExpression, + file.location()); + + if (allRowsMatch) { + writer.delete(entry); + + if (deletedFiles.contains(file)) { + LOG.warn( + "Deleting a duplicate path from manifest {}: {}", + manifest.path(), + file.location()); + duplicateDeleteCount += 1; } else { - writer.existing(entry); + // only add the file to deletes if it is a new delete + // this keeps the snapshot summary accurate for non-duplicate data + deletedFiles.add(file.copyWithoutStats()); } - } else { writer.existing(entry); } + + } else { + writer.existing(entry); } }); } finally { From 918f81f3c3f498f46afcea17c1ac9cdc6913cb5c Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 20 Nov 2024 15:06:33 +0100 Subject: [PATCH 164/313] Core: Fix CCE when retrieving TableOps (#11585) --- .../src/main/java/org/apache/iceberg/SerializableTable.java | 6 ++++++ .../org/apache/iceberg/hadoop/TestTableSerialization.java | 4 ++++ 2 files changed, 10 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index 082e50b840dc..a2c0d776423c 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -422,6 +422,12 @@ protected Table newTable(TableOperations ops, String tableName) { return MetadataTableUtils.createMetadataTableInstance(ops, baseTableName, tableName, type); } + @Override + public StaticTableOperations operations() { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not support operations()"); + } + public MetadataTableType type() { return type; } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java index a24c4245b157..5ef4697b4736 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java @@ -19,6 +19,7 @@ package org.apache.iceberg.hadoop; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -102,6 +103,9 @@ public void testSerializableMetadataTable() throws IOException, ClassNotFoundExc Table serializableTable = SerializableTable.copyOf(metadataTable); TestHelpers.assertSerializedAndLoadedMetadata( serializableTable, TestHelpers.KryoHelpers.roundTripSerialize(serializableTable)); + assertThatThrownBy(() -> ((HasTableOperations) serializableTable).operations()) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageEndingWith("does not support operations()"); } } From d19e3ff07653167d902865281601a5da4e2f2def Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 20 Nov 2024 17:32:11 +0100 Subject: [PATCH 165/313] API, Core: Remove unnecessary casts to Iterable (#11601) --- .../java/org/apache/iceberg/expressions/ExpressionUtil.java | 6 +++--- .../org/apache/iceberg/expressions/ExpressionParser.java | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java index 9be144b2ce98..68b9e9447926 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -337,9 +337,9 @@ public Expression predicate(UnboundPredicate pred) { pred.op(), pred.term(), (T) sanitize(pred.literal(), now, today)); case IN: case NOT_IN: - Iterable iter = - () -> pred.literals().stream().map(lit -> sanitize(lit, now, today)).iterator(); - return new UnboundPredicate<>(pred.op(), pred.term(), (Iterable) iter); + Iterable iter = + () -> pred.literals().stream().map(lit -> (T) sanitize(lit, now, today)).iterator(); + return new UnboundPredicate<>(pred.op(), pred.term(), iter); default: throw new UnsupportedOperationException( "Cannot sanitize unsupported predicate type: " + pred.op()); diff --git a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java index 9a6bfe863f0c..9bb5b7d05f0b 100644 --- a/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java +++ b/core/src/main/java/org/apache/iceberg/expressions/ExpressionParser.java @@ -352,8 +352,8 @@ private static UnboundPredicate predicateFromJson( node.has(VALUE), "Cannot parse %s predicate: missing value", op); Preconditions.checkArgument( !node.has(VALUES), "Cannot parse %s predicate: has invalid values field", op); - Object value = literal(JsonUtil.get(VALUE, node), convertValue); - return Expressions.predicate(op, term, (Iterable) ImmutableList.of(value)); + T value = literal(JsonUtil.get(VALUE, node), convertValue); + return Expressions.predicate(op, term, ImmutableList.of(value)); case IN: case NOT_IN: // literal set predicates From 799925a4ef41e7b4231930377b83bd686001c2c0 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 21 Nov 2024 00:39:41 +0800 Subject: [PATCH 166/313] Spark 3.5: Fix NotSerializableException when migrating Spark tables (#11157) --- .../iceberg/data/TableMigrationUtil.java | 2 + .../extensions/TestAddFilesProcedure.java | 20 +++ .../extensions/TestMigrateTableProcedure.java | 18 +++ .../TestSnapshotTableProcedure.java | 18 +++ .../apache/iceberg/spark/SparkTableUtil.java | 117 +++++++++++++++++- .../procedures/MigrateTableProcedure.java | 3 +- .../procedures/SnapshotTableProcedure.java | 3 +- 7 files changed, 177 insertions(+), 4 deletions(-) diff --git a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java index 0602c9e494c7..eb1c1a341e2b 100644 --- a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java +++ b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; +import javax.annotation.Nullable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -263,6 +264,7 @@ private static DataFile buildDataFile( *

    Important: Callers are responsible for shutting down the returned executor service * when it is no longer needed to prevent resource leaks. */ + @Nullable public static ExecutorService migrationService(int parallelism) { return parallelism == 1 ? null : ThreadPools.newFixedThreadPool("table-migration", parallelism); } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 920c2f55eaaf..332669470aea 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -948,6 +948,26 @@ public void testAddFilesWithParallelism() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @TestTemplate + public void testAddFilesPartitionedWithParallelism() { + createPartitionedHiveTable(); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + List result = + sql( + "CALL %s.system.add_files(table => '%s', source_table => '%s', parallelism => 2)", + catalogName, tableName, sourceTableName); + + assertOutput(result, 8L, 4L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + private static final List EMPTY_QUERY_RESULT = Lists.newArrayList(); private static final StructField[] STRUCT = { diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index 23c08b2572f4..69e80026e611 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -273,4 +273,22 @@ public void testMigrateWithInvalidParallelism() throws IOException { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Parallelism should be larger than 0"); } + + @TestTemplate + public void testMigratePartitionedWithParallelism() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s (id, data) VALUES (1, 'a'), (2, 'b')", tableName); + List result = + sql("CALL %s.system.migrate(table => '%s', parallelism => %d)", catalogName, tableName, 2); + assertEquals("Procedure output must match", ImmutableList.of(row(2L)), result); + assertEquals( + "Should have expected rows", + ImmutableList.of(row("a", 1L), row("b", 2L)), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 6caff28bb16c..28ae31ec6aa2 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -263,4 +263,22 @@ public void testSnapshotWithInvalidParallelism() throws IOException { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Parallelism should be larger than 0"); } + + @TestTemplate + public void testSnapshotPartitionedWithParallelism() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s (id, data) VALUES (1, 'a'), (2, 'b')", SOURCE_NAME); + List result = + sql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s', parallelism => %d)", + catalogName, SOURCE_NAME, tableName, 2); + assertEquals("Procedure output must match", ImmutableList.of(row(2L)), result); + assertEquals( + "Should have expected rows", + ImmutableList.of(row("a", 1L), row("b", 2L)), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index c44969c49e39..01912c81ccdf 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -23,12 +23,18 @@ import java.io.IOException; import java.io.Serializable; import java.net.URI; +import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -92,6 +98,8 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import scala.Function2; import scala.Option; import scala.Some; @@ -487,7 +495,7 @@ public static void importSparkTable( stagingDir, partitionFilter, checkDuplicateFiles, - TableMigrationUtil.migrationService(parallelism)); + migrationService(parallelism)); } /** @@ -711,7 +719,7 @@ public static void importSparkPartitions( spec, stagingDir, checkDuplicateFiles, - TableMigrationUtil.migrationService(parallelism)); + migrationService(parallelism)); } /** @@ -971,4 +979,109 @@ public int hashCode() { return Objects.hashCode(values, uri, format); } } + + @Nullable + public static ExecutorService migrationService(int parallelism) { + return parallelism == 1 ? null : new LazyExecutorService(parallelism); + } + + private static class LazyExecutorService implements ExecutorService, Serializable { + + private final int parallelism; + private volatile ExecutorService service; + + LazyExecutorService(int parallelism) { + this.parallelism = parallelism; + } + + @Override + public void shutdown() { + getService().shutdown(); + } + + @NotNull + @Override + public List shutdownNow() { + return getService().shutdownNow(); + } + + @Override + public boolean isShutdown() { + return getService().isShutdown(); + } + + @Override + public boolean isTerminated() { + return getService().isTerminated(); + } + + @Override + public boolean awaitTermination(long timeout, @NotNull TimeUnit unit) + throws InterruptedException { + return getService().awaitTermination(timeout, unit); + } + + @NotNull + @Override + public Future submit(@NotNull Callable task) { + return getService().submit(task); + } + + @NotNull + @Override + public Future submit(@NotNull Runnable task, T result) { + return getService().submit(task, result); + } + + @NotNull + @Override + public Future submit(@NotNull Runnable task) { + return getService().submit(task); + } + + @NotNull + @Override + public List> invokeAll(@NotNull Collection> tasks) + throws InterruptedException { + return getService().invokeAll(tasks); + } + + @NotNull + @Override + public List> invokeAll( + @NotNull Collection> tasks, long timeout, @NotNull TimeUnit unit) + throws InterruptedException { + return getService().invokeAll(tasks, timeout, unit); + } + + @NotNull + @Override + public T invokeAny(@NotNull Collection> tasks) + throws InterruptedException, ExecutionException { + return getService().invokeAny(tasks); + } + + @Override + public T invokeAny( + @NotNull Collection> tasks, long timeout, @NotNull TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { + return getService().invokeAny(tasks, timeout, unit); + } + + @Override + public void execute(@NotNull Runnable command) { + getService().execute(command); + } + + private ExecutorService getService() { + if (service == null) { + synchronized (this) { + if (service == null) { + service = TableMigrationUtil.migrationService(parallelism); + } + } + } + return service; + } + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java index a0bd04dd997e..7c67a1aceda3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java @@ -22,6 +22,7 @@ import org.apache.iceberg.actions.MigrateTable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.actions.MigrateTableSparkAction; import org.apache.iceberg.spark.actions.SparkActions; import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; @@ -110,7 +111,7 @@ public InternalRow[] call(InternalRow args) { int parallelism = args.getInt(4); Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); migrateTableSparkAction = - migrateTableSparkAction.executeWith(executorService(parallelism, "table-migration")); + migrateTableSparkAction.executeWith(SparkTableUtil.migrationService(parallelism)); } MigrateTable.Result result = migrateTableSparkAction.execute(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java index f709f64ebf62..37dfde76b7a7 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java @@ -22,6 +22,7 @@ import org.apache.iceberg.actions.SnapshotTable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.actions.SparkActions; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -106,7 +107,7 @@ public InternalRow[] call(InternalRow args) { if (!args.isNullAt(4)) { int parallelism = args.getInt(4); Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); - action = action.executeWith(executorService(parallelism, "table-snapshot")); + action = action.executeWith(SparkTableUtil.migrationService(parallelism)); } SnapshotTable.Result result = action.tableProperties(properties).execute(); From c9ece121446250643cb5e21133bef25522d69093 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 20 Nov 2024 10:40:48 -0800 Subject: [PATCH 167/313] Spark 3.3: Deprecate support (#11596) --- site/docs/multi-engine-support.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index ac17c29af546..ab20d59ff4f7 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -67,7 +67,7 @@ Each engine version undergoes the following lifecycle stages: | 3.0 | End of Life | 0.9.0 | 1.0.0 | [iceberg-spark-runtime-3.0_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.0_2.12/1.0.0/iceberg-spark-runtime-3.0_2.12-1.0.0.jar) | | 3.1 | End of Life | 0.12.0 | 1.3.1 | [iceberg-spark-runtime-3.1_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.1_2.12/1.3.1/iceberg-spark-runtime-3.1_2.12-1.3.1.jar) [1] | | 3.2 | End of Life | 0.13.0 | 1.4.3 | [iceberg-spark-runtime-3.2_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.2_2.12/1.4.3/iceberg-spark-runtime-3.2_2.12-1.4.3.jar) | -| 3.3 | Maintained | 0.14.0 | {{ icebergVersion }} | [iceberg-spark-runtime-3.3_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.3_2.12/{{ icebergVersion }}/iceberg-spark-runtime-3.3_2.12-{{ icebergVersion }}.jar) | +| 3.3 | Deprecated | 0.14.0 | {{ icebergVersion }} | [iceberg-spark-runtime-3.3_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.3_2.12/{{ icebergVersion }}/iceberg-spark-runtime-3.3_2.12-{{ icebergVersion }}.jar) | | 3.4 | Maintained | 1.3.0 | {{ icebergVersion }} | [iceberg-spark-runtime-3.4_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.4_2.12/{{ icebergVersion }}/iceberg-spark-runtime-3.4_2.12-{{ icebergVersion }}.jar) | | 3.5 | Maintained | 1.4.0 | {{ icebergVersion }} | [iceberg-spark-runtime-3.5_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/{{ icebergVersion }}/iceberg-spark-runtime-3.5_2.12-{{ icebergVersion }}.jar) | From 6e9e07aa0f35197bb23b218d23716e928d1c2814 Mon Sep 17 00:00:00 2001 From: Zhendong Bai Date: Thu, 21 Nov 2024 05:26:19 +0800 Subject: [PATCH 168/313] Hive: Bugfix for incorrect Deletion of Snapshot Metadata Due to OutOfMemoryError (#11576) --- .../iceberg/hive/HiveTableOperations.java | 1 + .../iceberg/hive/HiveViewOperations.java | 2 + .../apache/iceberg/hive/TestHiveCommits.java | 55 ++++++++++++++++++ .../iceberg/hive/TestHiveViewCommits.java | 58 +++++++++++++++++++ 4 files changed, 116 insertions(+) diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index 518daaf6acd1..619f20ab87a3 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -292,6 +292,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { database, tableName, e); + commitStatus = BaseMetastoreOperations.CommitStatus.UNKNOWN; commitStatus = BaseMetastoreOperations.CommitStatus.valueOf( checkCommitStatus(newMetadataLocation, metadata).name()); diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java index 4fc71299d457..dd0d5b3132d0 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.api.InvalidObjectException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.BaseMetastoreOperations; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.ClientPool; @@ -226,6 +227,7 @@ public void doCommit(ViewMetadata base, ViewMetadata metadata) { database, viewName, e); + commitStatus = BaseMetastoreOperations.CommitStatus.UNKNOWN; commitStatus = checkCommitStatus( viewName, diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java index 136c96934189..754ed55e81e8 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.HIVE_LOCK_ENABLED; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.doAnswer; @@ -44,6 +45,7 @@ import org.apache.iceberg.types.Types; import org.apache.thrift.TException; import org.junit.jupiter.api.Test; +import org.junit.platform.commons.support.ReflectionSupport; public class TestHiveCommits extends HiveTableBaseTest { @@ -399,6 +401,59 @@ public void testLockExceptionUnknownSuccessCommit() throws TException, Interrupt .isTrue(); } + @Test + public void testSuccessCommitWhenCheckCommitStatusOOM() throws TException, InterruptedException { + Table table = catalog.loadTable(TABLE_IDENTIFIER); + HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations(); + + TableMetadata metadataV1 = ops.current(); + + table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); + + ops.refresh(); + + TableMetadata metadataV2 = ops.current(); + + assertThat(ops.current().schema().columns()).hasSize(2); + + HiveTableOperations spyOps = spy(ops); + + // Simulate a communication error after a successful commit + doAnswer( + i -> { + org.apache.hadoop.hive.metastore.api.Table tbl = + i.getArgument(0, org.apache.hadoop.hive.metastore.api.Table.class); + String location = i.getArgument(2, String.class); + ops.persistTable(tbl, true, location); + throw new UnknownError(); + }) + .when(spyOps) + .persistTable(any(), anyBoolean(), any()); + try { + ReflectionSupport.invokeMethod( + ops.getClass() + .getSuperclass() + .getDeclaredMethod("checkCommitStatus", String.class, TableMetadata.class), + doThrow(new OutOfMemoryError()).when(spyOps), + anyString(), + any()); + } catch (Exception e) { + throw new RuntimeException(e); + } + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(OutOfMemoryError.class); + + ops.refresh(); + + assertThat(ops.current().location()) + .as("Current metadata should have changed to metadata V1") + .isEqualTo(metadataV1.location()); + assertThat(metadataFileExists(ops.current())) + .as("Current metadata file should still exist") + .isTrue(); + } + @Test public void testCommitExceptionWithoutMessage() throws TException, InterruptedException { Table table = catalog.loadTable(TABLE_IDENTIFIER); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java index 47abb51602fa..ae251aacebca 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.doAnswer; @@ -31,8 +32,10 @@ import java.io.File; import java.io.IOException; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; import org.apache.hadoop.fs.Path; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; @@ -53,6 +56,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.platform.commons.support.ReflectionSupport; /** Test Hive locks and Hive errors and retry during commits. */ public class TestHiveViewCommits { @@ -434,6 +438,60 @@ public void testLockExceptionUnknownSuccessCommit() throws TException, Interrupt assertThat(metadataFileCount(metadataV2)).as("New metadata file should exist").isEqualTo(2); } + @Test + public void testSuccessCommitWhenCheckCommitStatusOOM() throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + ViewMetadata metadataV2 = ops.current(); + assertThat(metadataV2.properties()).hasSize(1).containsEntry("k1", "v1"); + + HiveViewOperations spyOps = spy(ops); + + // Simulate a communication error after a successful commit + doAnswer( + i -> { + org.apache.hadoop.hive.metastore.api.Table tbl = + i.getArgument(0, org.apache.hadoop.hive.metastore.api.Table.class); + String location = i.getArgument(2, String.class); + ops.persistTable(tbl, true, location); + throw new UnknownError(); + }) + .when(spyOps) + .persistTable(any(), anyBoolean(), any()); + try { + ReflectionSupport.invokeMethod( + ops.getClass() + .getSuperclass() + .getSuperclass() + .getDeclaredMethod( + "checkCommitStatus", String.class, String.class, Map.class, Supplier.class), + doThrow(new OutOfMemoryError()).when(spyOps), + anyString(), + anyString(), + any(), + any()); + } catch (Exception e) { + throw new RuntimeException(e); + } + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(OutOfMemoryError.class); + + ops.refresh(); + + assertThat(metadataV2.location()) + .as("Current metadata should have changed to metadata V1") + .isEqualTo(metadataV1.location()); + assertThat(metadataFileExists(metadataV2)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(metadataV2)).as("New metadata file should exist").isEqualTo(2); + } + @Test public void testCommitExceptionWithoutMessage() throws TException, InterruptedException { HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); From a8f42d1b3954fb53a40204f2aa3a99080d637201 Mon Sep 17 00:00:00 2001 From: Matt Topol Date: Wed, 20 Nov 2024 17:08:36 -0500 Subject: [PATCH 169/313] docs: Add `iceberg-go` to doc site (#11607) --- docs/mkdocs.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index c1c16dc50096..87e3fdd8adf9 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -79,3 +79,4 @@ nav: - Javadoc: ../../javadoc/latest/ - PyIceberg: https://py.iceberg.apache.org/ - IcebergRust: https://rust.iceberg.apache.org/ + - IcebergGo: https://pkg.go.dev/github.com/apache/iceberg-go/ From 3b5c9f7f37b269c5e2df0feb11376333f29380e2 Mon Sep 17 00:00:00 2001 From: Karuppayya Date: Wed, 20 Nov 2024 15:10:26 -0800 Subject: [PATCH 170/313] Spark 3.5: Procedure to compute table stats (#10986) --- .../TestComputeTableStatsProcedure.java | 137 ++++++++++++++++++ .../ComputeTableStatsProcedure.java | 122 ++++++++++++++++ .../spark/procedures/SparkProcedures.java | 1 + 3 files changed, 260 insertions(+) create mode 100644 spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java new file mode 100644 index 000000000000..c487d4b7c848 --- /dev/null +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.BlobMetadata; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.actions.NDVSketchUtil; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestComputeTableStatsProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testProcedureOnEmptyTable() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + List result = + sql("CALL %s.system.compute_table_stats('%s')", catalogName, tableIdent); + assertThat(result).isEmpty(); + } + + @TestTemplate + public void testProcedureWithNamedArgs() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + List output = + sql( + "CALL %s.system.compute_table_stats(table => '%s', columns => array('id'))", + catalogName, tableIdent); + assertThat(output.get(0)).isNotEmpty(); + Object obj = output.get(0)[0]; + assertThat(obj.toString()).endsWith(".stats"); + verifyTableStats(tableName); + } + + @TestTemplate + public void testProcedureWithPositionalArgs() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Snapshot snapshot = table.currentSnapshot(); + List output = + sql( + "CALL %s.system.compute_table_stats('%s', %dL)", + catalogName, tableIdent, snapshot.snapshotId()); + assertThat(output.get(0)).isNotEmpty(); + Object obj = output.get(0)[0]; + assertThat(obj.toString()).endsWith(".stats"); + verifyTableStats(tableName); + } + + @TestTemplate + public void testProcedureWithInvalidColumns() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_table_stats(table => '%s', columns => array('id1'))", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can't find column id1"); + } + + @TestTemplate + public void testProcedureWithInvalidSnapshot() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_table_stats(table => '%s', snapshot_id => %dL)", + catalogName, tableIdent, 1234L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Snapshot not found"); + } + + @TestTemplate + public void testProcedureWithInvalidTable() { + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_table_stats(table => '%s', snapshot_id => %dL)", + catalogName, TableIdentifier.of(Namespace.of("default"), "abcd"), 1234L)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Couldn't load table"); + } + + void verifyTableStats(String tableName) throws NoSuchTableException, ParseException { + Table table = Spark3Util.loadIcebergTable(spark, tableName); + StatisticsFile statisticsFile = table.statisticsFiles().get(0); + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties()) + .containsKey(NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java new file mode 100644 index 000000000000..1c2d7125a38a --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.procedures; + +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.ComputeTableStats.Result; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter; +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.apache.spark.unsafe.types.UTF8String; + +/** + * A procedure that computes statistics of a table. + * + * @see SparkActions#computeTableStats(Table) + */ +public class ComputeTableStatsProcedure extends BaseProcedure { + + private static final ProcedureParameter TABLE_PARAM = + ProcedureParameter.required("table", DataTypes.StringType); + private static final ProcedureParameter SNAPSHOT_ID_PARAM = + ProcedureParameter.optional("snapshot_id", DataTypes.LongType); + private static final ProcedureParameter COLUMNS_PARAM = + ProcedureParameter.optional("columns", STRING_ARRAY); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, SNAPSHOT_ID_PARAM, COLUMNS_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("statistics_file", DataTypes.StringType, true, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new Builder() { + @Override + protected ComputeTableStatsProcedure doBuild() { + return new ComputeTableStatsProcedure(tableCatalog()); + } + }; + } + + private ComputeTableStatsProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + Long snapshotId = input.asLong(SNAPSHOT_ID_PARAM, null); + String[] columns = input.asStringArray(COLUMNS_PARAM, null); + + return modifyIcebergTable( + tableIdent, + table -> { + ComputeTableStats action = actions().computeTableStats(table); + + if (snapshotId != null) { + action.snapshot(snapshotId); + } + + if (columns != null) { + action.columns(columns); + } + + Result result = action.execute(); + return toOutputRows(result); + }); + } + + private InternalRow[] toOutputRows(Result result) { + StatisticsFile statisticsFile = result.statisticsFile(); + if (statisticsFile != null) { + InternalRow row = newInternalRow(UTF8String.fromString(statisticsFile.path())); + return new InternalRow[] {row}; + } else { + return new InternalRow[0]; + } + } + + @Override + public String description() { + return "ComputeTableStatsProcedure"; + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java index 42003b24e94c..d636a21ddc00 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java @@ -61,6 +61,7 @@ private static Map> initProcedureBuilders() { mapBuilder.put("create_changelog_view", CreateChangelogViewProcedure::builder); mapBuilder.put("rewrite_position_delete_files", RewritePositionDeleteFilesProcedure::builder); mapBuilder.put("fast_forward", FastForwardBranchProcedure::builder); + mapBuilder.put("compute_table_stats", ComputeTableStatsProcedure::builder); return mapBuilder.build(); } From 93a06339992a002fa907f515e68f4c20a3af2253 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 20 Nov 2024 17:46:57 -0800 Subject: [PATCH 171/313] Parquet: Use native getRowIndexOffset support instead of calculating it (#11520) --- .../arrow/vectorized/BaseBatchReader.java | 8 ++- .../vectorized/VectorizedArrowReader.java | 31 ++++++++++- .../apache/iceberg/parquet/ParquetReader.java | 5 +- .../iceberg/parquet/ParquetValueReader.java | 10 ++++ .../iceberg/parquet/ParquetValueReaders.java | 54 ++++++++++++++++--- .../org/apache/iceberg/parquet/ReadConf.java | 44 --------------- .../parquet/VectorizedParquetReader.java | 5 +- .../iceberg/parquet/VectorizedReader.java | 15 ++++++ .../data/vectorized/ColumnarBatchReader.java | 16 +++++- .../data/vectorized/ColumnarBatchReader.java | 16 +++++- .../data/vectorized/ColumnarBatchReader.java | 16 +++++- 11 files changed, 154 insertions(+), 66 deletions(-) diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/BaseBatchReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/BaseBatchReader.java index 2175293ab2b6..daa116f292ed 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/BaseBatchReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/BaseBatchReader.java @@ -42,9 +42,15 @@ protected BaseBatchReader(List> readers) { @Override public void setRowGroupInfo( PageReadStore pageStore, Map metaData, long rowPosition) { + setRowGroupInfo(pageStore, metaData); + } + + @Override + public void setRowGroupInfo( + PageReadStore pageStore, Map metaData) { for (VectorizedArrowReader reader : readers) { if (reader != null) { - reader.setRowGroupInfo(pageStore, metaData, rowPosition); + reader.setRowGroupInfo(pageStore, metaData); } } } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java index 27ee25124f16..411f241e169f 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java @@ -432,6 +432,11 @@ private void allocateVectorBasedOnTypeName(PrimitiveType primitive, Field arrowF @Override public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) { + setRowGroupInfo(source, metadata); + } + + @Override + public void setRowGroupInfo(PageReadStore source, Map metadata) { ColumnChunkMetaData chunkMetaData = metadata.get(ColumnPath.get(columnDescriptor.getPath())); this.dictionary = vectorizedColumnIterator.setRowGroupInfo( @@ -475,6 +480,10 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) { public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) {} + @Override + public void setRowGroupInfo( + PageReadStore source, Map metadata) {} + @Override public String toString() { return "NullReader"; @@ -541,7 +550,19 @@ private static NullabilityHolder newNullabilityHolder(int size) { @Override public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) { - this.rowStart = rowPosition; + setRowGroupInfo(source, metadata); + } + + @Override + public void setRowGroupInfo( + PageReadStore source, Map metadata) { + this.rowStart = + source + .getRowIndexOffset() + .orElseThrow( + () -> + new IllegalArgumentException( + "PageReadStore does not contain row index offset")); } @Override @@ -586,6 +607,10 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) { public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) {} + @Override + public void setRowGroupInfo( + PageReadStore source, Map metadata) {} + @Override public String toString() { return String.format("ConstantReader: %s", value); @@ -613,6 +638,10 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) { public void setRowGroupInfo( PageReadStore source, Map metadata, long rowPosition) {} + @Override + public void setRowGroupInfo( + PageReadStore source, Map metadata) {} + @Override public String toString() { return "DeletedVectorReader"; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java index c1d8b0ccbbad..e8ee90fdebb7 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java @@ -99,7 +99,6 @@ private static class FileIterator implements CloseableIterator { private final ParquetValueReader model; private final long totalValues; private final boolean reuseContainers; - private final long[] rowGroupsStartRowPos; private int nextRowGroup = 0; private long nextRowGroupStart = 0; @@ -112,7 +111,6 @@ private static class FileIterator implements CloseableIterator { this.model = conf.model(); this.totalValues = conf.totalValues(); this.reuseContainers = conf.reuseContainers(); - this.rowGroupsStartRowPos = conf.startRowPositions(); } @Override @@ -149,11 +147,10 @@ private void advance() { throw new RuntimeIOException(e); } - long rowPosition = rowGroupsStartRowPos[nextRowGroup]; nextRowGroupStart += pages.getRowCount(); nextRowGroup += 1; - model.setPageSource(pages, rowPosition); + model.setPageSource(pages); } @Override diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java index b6c2b5b70303..a2ade5336621 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java @@ -28,5 +28,15 @@ public interface ParquetValueReader { List> columns(); + /** + * @deprecated since 1.8.0, will be removed in 1.9.0; use {@link #setPageSource(PageReadStore)} + * instead. + */ + @Deprecated void setPageSource(PageReadStore pageStore, long rowPosition); + + default void setPageSource(PageReadStore pageStore) { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement setPageSource(PageReadStore)"); + } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java index 62a49da25e56..b055a139fa59 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java @@ -113,6 +113,9 @@ public List> columns() { @Override public void setPageSource(PageReadStore pageStore, long rowPosition) {} + + @Override + public void setPageSource(PageReadStore pageStore) {} } static class ConstantReader implements ParquetValueReader { @@ -176,6 +179,9 @@ public List> columns() { @Override public void setPageSource(PageReadStore pageStore, long rowPosition) {} + + @Override + public void setPageSource(PageReadStore pageStore) {} } static class PositionReader implements ParquetValueReader { @@ -200,7 +206,18 @@ public List> columns() { @Override public void setPageSource(PageReadStore pageStore, long rowPosition) { - this.rowGroupStart = rowPosition; + setPageSource(pageStore); + } + + @Override + public void setPageSource(PageReadStore pageStore) { + this.rowGroupStart = + pageStore + .getRowIndexOffset() + .orElseThrow( + () -> + new IllegalArgumentException( + "PageReadStore does not contain row index offset")); this.rowOffset = -1; } } @@ -221,6 +238,11 @@ protected PrimitiveReader(ColumnDescriptor desc) { @Override public void setPageSource(PageReadStore pageStore, long rowPosition) { + setPageSource(pageStore); + } + + @Override + public void setPageSource(PageReadStore pageStore) { column.setPageSource(pageStore.getPageReader(desc)); } @@ -405,7 +427,12 @@ private static class OptionReader implements ParquetValueReader { @Override public void setPageSource(PageReadStore pageStore, long rowPosition) { - reader.setPageSource(pageStore, rowPosition); + setPageSource(pageStore); + } + + @Override + public void setPageSource(PageReadStore pageStore) { + reader.setPageSource(pageStore); } @Override @@ -450,7 +477,12 @@ protected RepeatedReader( @Override public void setPageSource(PageReadStore pageStore, long rowPosition) { - reader.setPageSource(pageStore, rowPosition); + setPageSource(pageStore); + } + + @Override + public void setPageSource(PageReadStore pageStore) { + reader.setPageSource(pageStore); } @Override @@ -569,8 +601,13 @@ protected RepeatedKeyValueReader( @Override public void setPageSource(PageReadStore pageStore, long rowPosition) { - keyReader.setPageSource(pageStore, rowPosition); - valueReader.setPageSource(pageStore, rowPosition); + setPageSource(pageStore); + } + + @Override + public void setPageSource(PageReadStore pageStore) { + keyReader.setPageSource(pageStore); + valueReader.setPageSource(pageStore); } @Override @@ -720,8 +757,13 @@ protected StructReader(List types, List> readers) { @Override public final void setPageSource(PageReadStore pageStore, long rowPosition) { + setPageSource(pageStore); + } + + @Override + public final void setPageSource(PageReadStore pageStore) { for (ParquetValueReader reader : readers) { - reader.setPageSource(pageStore, rowPosition); + reader.setPageSource(pageStore); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index da91e4dfa56a..1fb2372ba568 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -19,13 +19,11 @@ package org.apache.iceberg.parquet; import java.io.IOException; -import java.io.UncheckedIOException; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Expression; @@ -33,9 +31,7 @@ import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.crypto.FileDecryptionProperties; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; @@ -59,7 +55,6 @@ class ReadConf { private final long totalValues; private final boolean reuseContainers; private final Integer batchSize; - private final long[] startRowPositions; // List of column chunk metadata for each row group private final List> columnChunkMetaDataForRowGroups; @@ -95,10 +90,6 @@ class ReadConf { this.rowGroups = reader.getRowGroups(); this.shouldSkip = new boolean[rowGroups.size()]; - this.startRowPositions = new long[rowGroups.size()]; - - // Fetch all row groups starting positions to compute the row offsets of the filtered row groups - Map offsetToStartPos = generateOffsetToStartPos(expectedSchema); ParquetMetricsRowGroupFilter statsFilter = null; ParquetDictionaryRowGroupFilter dictFilter = null; @@ -112,8 +103,6 @@ class ReadConf { long computedTotalValues = 0L; for (int i = 0; i < shouldSkip.length; i += 1) { BlockMetaData rowGroup = rowGroups.get(i); - startRowPositions[i] = - offsetToStartPos == null ? 0 : offsetToStartPos.get(rowGroup.getStartingPos()); boolean shouldRead = filter == null || (statsFilter.shouldRead(typeWithIds, rowGroup) @@ -155,7 +144,6 @@ private ReadConf(ReadConf toCopy) { this.batchSize = toCopy.batchSize; this.vectorizedModel = toCopy.vectorizedModel; this.columnChunkMetaDataForRowGroups = toCopy.columnChunkMetaDataForRowGroups; - this.startRowPositions = toCopy.startRowPositions; } ParquetFileReader reader() { @@ -181,38 +169,6 @@ boolean[] shouldSkip() { return shouldSkip; } - private Map generateOffsetToStartPos(Schema schema) { - if (schema.findField(MetadataColumns.ROW_POSITION.fieldId()) == null) { - return null; - } - - FileDecryptionProperties decryptionProperties = - (options == null) ? null : options.getDecryptionProperties(); - - ParquetReadOptions readOptions = - ParquetReadOptions.builder().withDecryption(decryptionProperties).build(); - - try (ParquetFileReader fileReader = newReader(file, readOptions)) { - Map offsetToStartPos = Maps.newHashMap(); - - long curRowCount = 0; - for (int i = 0; i < fileReader.getRowGroups().size(); i += 1) { - BlockMetaData meta = fileReader.getRowGroups().get(i); - offsetToStartPos.put(meta.getStartingPos(), curRowCount); - curRowCount += meta.getRowCount(); - } - - return offsetToStartPos; - - } catch (IOException e) { - throw new UncheckedIOException("Failed to create/close reader for file: " + file, e); - } - } - - long[] startRowPositions() { - return startRowPositions; - } - long totalValues() { return totalValues; } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java index 35d94f328d60..fc10a57ec0e0 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java @@ -113,7 +113,6 @@ private static class FileIterator implements CloseableIterator { private long nextRowGroupStart = 0; private long valuesRead = 0; private T last = null; - private final long[] rowGroupsStartRowPos; FileIterator(ReadConf conf) { this.reader = conf.reader(); @@ -124,7 +123,6 @@ private static class FileIterator implements CloseableIterator { this.batchSize = conf.batchSize(); this.model.setBatchSize(this.batchSize); this.columnChunkMetadata = conf.columnChunkMetadataForRowGroups(); - this.rowGroupsStartRowPos = conf.startRowPositions(); } @Override @@ -165,8 +163,7 @@ private void advance() { throw new RuntimeIOException(e); } - long rowPosition = rowGroupsStartRowPos[nextRowGroup]; - model.setRowGroupInfo(pages, columnChunkMetadata.get(nextRowGroup), rowPosition); + model.setRowGroupInfo(pages, columnChunkMetadata.get(nextRowGroup)); nextRowGroupStart += pages.getRowCount(); nextRowGroup += 1; } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedReader.java index 72b1e39e9634..caf2b6ff22e8 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedReader.java @@ -43,10 +43,25 @@ public interface VectorizedReader { * @param pages row group information for all the columns * @param metadata map of {@link ColumnPath} -> {@link ColumnChunkMetaData} for the row group * @param rowPosition the row group's row offset in the parquet file + * @deprecated since 1.8.0, will be removed in 1.9.0; use {@link #setRowGroupInfo(PageReadStore, + * Map)} instead. */ + @Deprecated void setRowGroupInfo( PageReadStore pages, Map metadata, long rowPosition); + /** + * Sets the row group information to be used with this reader + * + * @param pages row group information for all the columns + * @param metadata map of {@link ColumnPath} -> {@link ColumnChunkMetaData} for the row group + */ + default void setRowGroupInfo(PageReadStore pages, Map metadata) { + throw new UnsupportedOperationException( + this.getClass().getName() + + " doesn't implement setRowGroupInfo(PageReadStore, Map)"); + } + /** Release any resources allocated. */ void close(); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java index f07d8c545e35..77cb2ff771c8 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java @@ -55,8 +55,20 @@ public ColumnarBatchReader(List> readers) { @Override public void setRowGroupInfo( PageReadStore pageStore, Map metaData, long rowPosition) { - super.setRowGroupInfo(pageStore, metaData, rowPosition); - this.rowStartPosInBatch = rowPosition; + setRowGroupInfo(pageStore, metaData); + } + + @Override + public void setRowGroupInfo( + PageReadStore pageStore, Map metaData) { + super.setRowGroupInfo(pageStore, metaData); + this.rowStartPosInBatch = + pageStore + .getRowIndexOffset() + .orElseThrow( + () -> + new IllegalArgumentException( + "PageReadStore does not contain row index offset")); } public void setDeleteFilter(DeleteFilter deleteFilter) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java index f07d8c545e35..77cb2ff771c8 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java @@ -55,8 +55,20 @@ public ColumnarBatchReader(List> readers) { @Override public void setRowGroupInfo( PageReadStore pageStore, Map metaData, long rowPosition) { - super.setRowGroupInfo(pageStore, metaData, rowPosition); - this.rowStartPosInBatch = rowPosition; + setRowGroupInfo(pageStore, metaData); + } + + @Override + public void setRowGroupInfo( + PageReadStore pageStore, Map metaData) { + super.setRowGroupInfo(pageStore, metaData); + this.rowStartPosInBatch = + pageStore + .getRowIndexOffset() + .orElseThrow( + () -> + new IllegalArgumentException( + "PageReadStore does not contain row index offset")); } public void setDeleteFilter(DeleteFilter deleteFilter) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java index f07d8c545e35..77cb2ff771c8 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java @@ -55,8 +55,20 @@ public ColumnarBatchReader(List> readers) { @Override public void setRowGroupInfo( PageReadStore pageStore, Map metaData, long rowPosition) { - super.setRowGroupInfo(pageStore, metaData, rowPosition); - this.rowStartPosInBatch = rowPosition; + setRowGroupInfo(pageStore, metaData); + } + + @Override + public void setRowGroupInfo( + PageReadStore pageStore, Map metaData) { + super.setRowGroupInfo(pageStore, metaData); + this.rowStartPosInBatch = + pageStore + .getRowIndexOffset() + .orElseThrow( + () -> + new IllegalArgumentException( + "PageReadStore does not contain row index offset")); } public void setDeleteFilter(DeleteFilter deleteFilter) { From c448a4b87007ae8aad57456e19d32db45de3a4f6 Mon Sep 17 00:00:00 2001 From: Ace Haidrey Date: Wed, 20 Nov 2024 21:15:44 -0500 Subject: [PATCH 172/313] Spark: Fix changelog table bug for start time older than current snapshot (#11564) --- .../spark/extensions/TestChangelogTable.java | 48 +++++++++++++++++++ .../spark/source/SparkScanBuilder.java | 4 ++ 2 files changed, 52 insertions(+) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java index a7ed065cae2c..d4930250cb10 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java @@ -408,4 +408,52 @@ private List collect(DataFrameReader reader) { .orderBy("_change_ordinal", "_commit_snapshot_id", "_change_type", "id") .collectAsList(); } + + @TestTemplate + public void testChangelogViewOutsideTimeRange() { + createTableWithDefaultRows(); + + // Insert new records + sql("INSERT INTO %s VALUES (3, 'c')", tableName); + sql("INSERT INTO %s VALUES (4, 'd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot insertSnapshot = table.currentSnapshot(); + + // Get timestamp after inserts but before our changelog window + long beforeWindowTime = System.currentTimeMillis(); + + // Small delay to ensure our timestamps are different + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException("Test interrupted", e); + } + + long startTime = System.currentTimeMillis(); + long endTime = startTime + 1000; // 1 second window + + // Create changelog view for a time window after our inserts + sql( + "CALL %s.system.create_changelog_view(" + + " table => '%s', " + + " options => map(" + + " 'start-timestamp', '%d'," + + " 'end-timestamp', '%d'" + + " )," + + " changelog_view => 'test_changelog_view'" + + ")", + catalogName, tableName, startTime, endTime); + + // Query the changelog view + List results = + sql( + "SELECT * FROM test_changelog_view WHERE _change_type IN ('INSERT', 'DELETE') ORDER BY _change_ordinal"); + + // Verify no changes are returned since our window is after the inserts + assertThat(results).as("Num records must be zero").isEmpty(); + + // Clean up the changelog view + sql("DROP VIEW IF EXISTS test_changelog_view"); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index d511fefd8ae0..33a4e032ccc8 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -561,6 +561,10 @@ public Scan buildChangelogScan() { boolean emptyScan = false; if (startTimestamp != null) { + if (table.currentSnapshot() != null + && table.currentSnapshot().timestampMillis() < startTimestamp) { + emptyScan = true; + } startSnapshotId = getStartSnapshotId(startTimestamp); if (startSnapshotId == null && endTimestamp == null) { emptyScan = true; From 652fcc66073f996af5515eca8a094886c2fd1312 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 21 Nov 2024 23:14:23 +0800 Subject: [PATCH 173/313] Spark 3.5: Fix flaky TestRemoveOrphanFilesAction3 (#11616) --- .../actions/TestRemoveOrphanFilesAction.java | 9 ++++++--- .../actions/TestRemoveOrphanFilesAction3.java | 19 +++++++++---------- 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index d36898d4c464..a0016a5e421a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -33,10 +33,10 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -726,8 +726,7 @@ public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException @TestTemplate public void testHiveCatalogTable() throws IOException { - TableIdentifier identifier = - TableIdentifier.of("default", "hivetestorphan" + ThreadLocalRandom.current().nextInt(1000)); + TableIdentifier identifier = TableIdentifier.of("default", randomName("hivetestorphan")); Table table = catalog.createTable(identifier, SCHEMA, SPEC, tableLocation, properties); List records = @@ -1068,6 +1067,10 @@ public void testRemoveOrphanFileActionWithDeleteMode() { DeleteOrphanFiles.PrefixMismatchMode.DELETE); } + protected String randomName(String prefix) { + return prefix + UUID.randomUUID().toString().replace("-", ""); + } + private void executeTest( List validFiles, List actualFiles, List expectedOrphanFiles) { executeTest( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java index 14784da4f74f..35d86b0a44b0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -21,7 +21,6 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.File; -import java.util.concurrent.ThreadLocalRandom; import java.util.stream.StreamSupport; import org.apache.iceberg.actions.DeleteOrphanFiles; import org.apache.iceberg.spark.SparkCatalog; @@ -42,7 +41,7 @@ public void testSparkCatalogTable() throws Exception { SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("mycat"); String[] database = {"default"}; - Identifier id = Identifier.of(database, "table" + ThreadLocalRandom.current().nextInt(1000)); + Identifier id = Identifier.of(database, randomName("table")); Transform[] transforms = {}; cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); SparkTable table = (SparkTable) cat.loadTable(id); @@ -50,7 +49,7 @@ public void testSparkCatalogTable() throws Exception { sql("INSERT INTO mycat.default.%s VALUES (1,1,1)", id.name()); String location = table.table().location().replaceFirst("file:", ""); - String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + String trashFile = randomName("/data/trashfile"); new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = @@ -71,7 +70,7 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("hadoop"); String[] database = {"default"}; - Identifier id = Identifier.of(database, "table" + ThreadLocalRandom.current().nextInt(1000)); + Identifier id = Identifier.of(database, randomName("table")); Transform[] transforms = {}; cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); SparkTable table = (SparkTable) cat.loadTable(id); @@ -79,7 +78,7 @@ public void testSparkCatalogNamedHadoopTable() throws Exception { sql("INSERT INTO hadoop.default.%s VALUES (1,1,1)", id.name()); String location = table.table().location().replaceFirst("file:", ""); - String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + String trashFile = randomName("/data/trashfile"); new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = @@ -100,7 +99,7 @@ public void testSparkCatalogNamedHiveTable() throws Exception { SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("hive"); String[] database = {"default"}; - Identifier id = Identifier.of(database, "table" + ThreadLocalRandom.current().nextInt(1000)); + Identifier id = Identifier.of(database, randomName("table")); Transform[] transforms = {}; cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); SparkTable table = (SparkTable) cat.loadTable(id); @@ -108,7 +107,7 @@ public void testSparkCatalogNamedHiveTable() throws Exception { sql("INSERT INTO hive.default.%s VALUES (1,1,1)", id.name()); String location = table.table().location().replaceFirst("file:", ""); - String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + String trashFile = randomName("/data/trashfile"); new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = @@ -133,7 +132,7 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { (SparkSessionCatalog) spark.sessionState().catalogManager().v2SessionCatalog(); String[] database = {"default"}; - Identifier id = Identifier.of(database, "table" + ThreadLocalRandom.current().nextInt(1000)); + Identifier id = Identifier.of(database, randomName("table")); Transform[] transforms = {}; cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); SparkTable table = (SparkTable) cat.loadTable(id); @@ -141,7 +140,7 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { sql("INSERT INTO default.%s VALUES (1,1,1)", id.name()); String location = table.table().location().replaceFirst("file:", ""); - String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + String trashFile = randomName("/data/trashfile"); new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = @@ -173,7 +172,7 @@ public void testSparkSessionCatalogHiveTable() throws Exception { spark.sql("INSERT INTO default.sessioncattest VALUES (1,1,1)"); String location = table.table().location().replaceFirst("file:", ""); - String trashFile = "/data/trashfile" + ThreadLocalRandom.current().nextInt(1000); + String trashFile = randomName("/data/trashfile"); new File(location + trashFile).createNewFile(); DeleteOrphanFiles.Result results = From c1f1f8b18e0fd1a36e170f05e9c7fa4f2bdd7b8d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Thu, 21 Nov 2024 16:15:42 +0100 Subject: [PATCH 174/313] Build: Upgrade to Gradle 8.11.1 (#11619) --- gradle/wrapper/gradle-wrapper.properties | 4 ++-- gradlew | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 82dd18b2043e..eb1a55be0e15 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,7 +1,7 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionSha256Sum=57dafb5c2622c6cc08b993c85b7c06956a2f53536432a30ead46166dbca0f1e9 -distributionUrl=https\://services.gradle.org/distributions/gradle-8.11-bin.zip +distributionSha256Sum=f397b287023acdba1e9f6fc5ea72d22dd63669d59ed4a289a29b1a76eee151c6 +distributionUrl=https\://services.gradle.org/distributions/gradle-8.11.1-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 79ec571bc6ae..4f63945674c2 100755 --- a/gradlew +++ b/gradlew @@ -87,7 +87,7 @@ APP_BASE_NAME=${0##*/} APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit if [ ! -e $APP_HOME/gradle/wrapper/gradle-wrapper.jar ]; then - curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.11.0/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.11.1/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. From 90be5d7360bc7ff274e7d00cb7259afbf78f223b Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 21 Nov 2024 08:23:53 -0700 Subject: [PATCH 175/313] Core: Optimize MergingSnapshotProducer to use referenced manifests to determine if manifest needs to be rewritten (#11131) --- .../iceberg/ReplaceDeleteFilesBenchmark.java | 54 +++++-- .../apache/iceberg/ManifestFilterManager.java | 105 ++++++++----- .../java/org/apache/iceberg/TestRowDelta.java | 144 ++++++++++++++++-- .../org/apache/iceberg/TestTransaction.java | 128 ++++++++++++++++ .../TestExpireSnapshotsProcedure.java | 12 +- .../TestExpireSnapshotsProcedure.java | 12 +- .../TestExpireSnapshotsProcedure.java | 12 +- 7 files changed, 390 insertions(+), 77 deletions(-) diff --git a/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java index a899b870a90c..1db327da8c8c 100644 --- a/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java @@ -20,10 +20,13 @@ import static org.apache.iceberg.types.Types.NestedField.required; +import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -70,14 +73,17 @@ public class ReplaceDeleteFilesBenchmark { private static final HadoopTables TABLES = new HadoopTables(); private Table table; - private List deleteFiles; + private List deleteFilesToReplace; private List pendingDeleteFiles; - @Param({"50000", "100000", "500000", "1000000", "2500000"}) + @Param({"50000", "100000", "500000", "1000000", "2000000"}) private int numFiles; + @Param({"5", "25", "50", "100"}) + private int percentDeleteFilesReplaced; + @Setup - public void setupBenchmark() { + public void setupBenchmark() throws IOException { initTable(); initFiles(); } @@ -90,10 +96,13 @@ public void tearDownBenchmark() { @Benchmark @Threads(1) public void replaceDeleteFiles() { + Snapshot currentSnapshot = table.currentSnapshot(); RowDelta rowDelta = table.newRowDelta(); - deleteFiles.forEach(rowDelta::removeDeletes); + rowDelta.validateFromSnapshot(currentSnapshot.snapshotId()); + deleteFilesToReplace.forEach(rowDelta::removeDeletes); pendingDeleteFiles.forEach(rowDelta::addDeletes); rowDelta.commit(); + table.manageSnapshots().rollbackTo(currentSnapshot.snapshotId()).commit(); } private void initTable() { @@ -104,27 +113,44 @@ private void dropTable() { TABLES.dropTable(TABLE_IDENT); } - private void initFiles() { - List generatedDeleteFiles = Lists.newArrayListWithExpectedSize(numFiles); + private void initFiles() throws IOException { List generatedPendingDeleteFiles = Lists.newArrayListWithExpectedSize(numFiles); - + int numDeleteFilesToReplace = (int) Math.ceil(numFiles * (percentDeleteFilesReplaced / 100.0)); + Map filesToReplace = + Maps.newHashMapWithExpectedSize(numDeleteFilesToReplace); RowDelta rowDelta = table.newRowDelta(); - for (int ordinal = 0; ordinal < numFiles; ordinal++) { DataFile dataFile = FileGenerationUtil.generateDataFile(table, null); rowDelta.addRows(dataFile); - DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, dataFile); rowDelta.addDeletes(deleteFile); - generatedDeleteFiles.add(deleteFile); - - DeleteFile pendingDeleteFile = FileGenerationUtil.generatePositionDeleteFile(table, dataFile); - generatedPendingDeleteFiles.add(pendingDeleteFile); + if (numDeleteFilesToReplace > 0) { + filesToReplace.put(deleteFile.location(), deleteFile); + DeleteFile pendingDeleteFile = + FileGenerationUtil.generatePositionDeleteFile(table, dataFile); + generatedPendingDeleteFiles.add(pendingDeleteFile); + numDeleteFilesToReplace--; + } } rowDelta.commit(); - this.deleteFiles = generatedDeleteFiles; + List deleteFilesReadFromManifests = Lists.newArrayList(); + for (ManifestFile deleteManifest : table.currentSnapshot().deleteManifests(table.io())) { + try (ManifestReader manifestReader = + ManifestFiles.readDeleteManifest(deleteManifest, table.io(), table.specs())) { + manifestReader + .iterator() + .forEachRemaining( + file -> { + if (filesToReplace.containsKey(file.location())) { + deleteFilesReadFromManifests.add(file); + } + }); + } + } + this.pendingDeleteFiles = generatedPendingDeleteFiles; + this.deleteFilesToReplace = deleteFilesReadFromManifests; } } diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index 518599b99c6b..840d93ec1055 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.ManifestFileUtil; import org.apache.iceberg.util.Pair; @@ -69,6 +70,7 @@ public String partition() { private final Map specsById; private final PartitionSet deleteFilePartitions; private final Set deleteFiles = newFileSet(); + private final Set manifestsWithDeletes = Sets.newHashSet(); private final PartitionSet dropPartitions; private final CharSequenceSet deletePaths = CharSequenceSet.empty(); private Expression deleteExpression = Expressions.alwaysFalse(); @@ -77,6 +79,7 @@ public String partition() { private boolean failMissingDeletePaths = false; private int duplicateDeleteCount = 0; private boolean caseSensitive = true; + private boolean allDeletesReferenceManifests = true; // cache filtered manifests to avoid extra work when commits fail. private final Map filteredManifests = Maps.newConcurrentMap(); @@ -121,6 +124,7 @@ protected void deleteByRowFilter(Expression expr) { Preconditions.checkNotNull(expr, "Cannot delete files using filter: null"); invalidateFilteredCache(); this.deleteExpression = Expressions.or(deleteExpression, expr); + this.allDeletesReferenceManifests = false; } /** Add a partition tuple to drop from the table during the delete phase. */ @@ -128,6 +132,7 @@ protected void dropPartition(int specId, StructLike partition) { Preconditions.checkNotNull(partition, "Cannot delete files in invalid partition: null"); invalidateFilteredCache(); dropPartitions.add(specId, partition); + this.allDeletesReferenceManifests = false; } /** @@ -154,6 +159,13 @@ void caseSensitive(boolean newCaseSensitive) { void delete(F file) { Preconditions.checkNotNull(file, "Cannot delete file: null"); invalidateFilteredCache(); + + if (file.manifestLocation() == null) { + this.allDeletesReferenceManifests = false; + } else { + manifestsWithDeletes.add(file.manifestLocation()); + } + deleteFiles.add(file); deleteFilePartitions.add(file.specId(), file.partition()); } @@ -162,6 +174,7 @@ void delete(F file) { void delete(CharSequence path) { Preconditions.checkNotNull(path, "Cannot delete file path: null"); invalidateFilteredCache(); + this.allDeletesReferenceManifests = false; deletePaths.add(path); } @@ -185,6 +198,7 @@ List filterManifests(Schema tableSchema, List manife return ImmutableList.of(); } + boolean trustManifestReferences = canTrustManifestReferences(manifests); ManifestFile[] filtered = new ManifestFile[manifests.size()]; // open all of the manifest files in parallel, use index to avoid reordering Tasks.range(filtered.length) @@ -193,7 +207,8 @@ List filterManifests(Schema tableSchema, List manife .executeWith(workerPoolSupplier.get()) .run( index -> { - ManifestFile manifest = filterManifest(tableSchema, manifests.get(index)); + ManifestFile manifest = + filterManifest(tableSchema, manifests.get(index), trustManifestReferences); filtered[index] = manifest; }); @@ -202,6 +217,16 @@ List filterManifests(Schema tableSchema, List manife return Arrays.asList(filtered); } + // Use the current set of referenced manifests as a source of truth when it's a subset of all + // manifests and all removals which were performed reference manifests. + // If a manifest without live files is not in the trusted referenced set, this means that the + // manifest has no deleted entries and does not need to be rewritten. + private boolean canTrustManifestReferences(List manifests) { + Set manifestLocations = + manifests.stream().map(ManifestFile::path).collect(Collectors.toSet()); + return allDeletesReferenceManifests && manifestLocations.containsAll(manifestsWithDeletes); + } + /** * Creates a snapshot summary builder with the files deleted from the set of filtered manifests. * @@ -307,14 +332,14 @@ private void invalidateFilteredCache() { /** * @return a ManifestReader that is a filtered version of the input manifest. */ - private ManifestFile filterManifest(Schema tableSchema, ManifestFile manifest) { + private ManifestFile filterManifest( + Schema tableSchema, ManifestFile manifest, boolean trustManifestReferences) { ManifestFile cached = filteredManifests.get(manifest); if (cached != null) { return cached; } - boolean hasLiveFiles = manifest.hasAddedFiles() || manifest.hasExistingFiles(); - if (!hasLiveFiles || !canContainDeletedFiles(manifest)) { + if (!canContainDeletedFiles(manifest, trustManifestReferences)) { filteredManifests.put(manifest, manifest); return manifest; } @@ -323,66 +348,74 @@ private ManifestFile filterManifest(Schema tableSchema, ManifestFile manifest) { PartitionSpec spec = reader.spec(); PartitionAndMetricsEvaluator evaluator = new PartitionAndMetricsEvaluator(tableSchema, spec, deleteExpression); - // this assumes that the manifest doesn't have files to remove and streams through the // manifest without copying data. if a manifest does have a file to remove, this will break // out of the loop and move on to filtering the manifest. - boolean hasDeletedFiles = manifestHasDeletedFiles(evaluator, reader); - if (!hasDeletedFiles) { + if (manifestHasDeletedFiles(evaluator, manifest, reader)) { + return filterManifestWithDeletedFiles(evaluator, manifest, reader); + } else { filteredManifests.put(manifest, manifest); return manifest; } - - return filterManifestWithDeletedFiles(evaluator, manifest, reader); - } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close manifest: %s", manifest); } } - private boolean canContainDeletedFiles(ManifestFile manifest) { - boolean canContainExpressionDeletes; + private boolean canContainDeletedFiles(ManifestFile manifest, boolean trustManifestReferences) { + if (hasNoLiveFiles(manifest)) { + return false; + } + + if (trustManifestReferences) { + return manifestsWithDeletes.contains(manifest.path()); + } + + return canContainDroppedFiles(manifest) + || canContainExpressionDeletes(manifest) + || canContainDroppedPartitions(manifest); + } + + private boolean hasNoLiveFiles(ManifestFile manifest) { + return !manifest.hasAddedFiles() && !manifest.hasExistingFiles(); + } + + private boolean canContainExpressionDeletes(ManifestFile manifest) { if (deleteExpression != null && deleteExpression != Expressions.alwaysFalse()) { ManifestEvaluator manifestEvaluator = ManifestEvaluator.forRowFilter( deleteExpression, specsById.get(manifest.partitionSpecId()), caseSensitive); - canContainExpressionDeletes = manifestEvaluator.eval(manifest); - } else { - canContainExpressionDeletes = false; + return manifestEvaluator.eval(manifest); } - boolean canContainDroppedPartitions; + return false; + } + + private boolean canContainDroppedPartitions(ManifestFile manifest) { if (!dropPartitions.isEmpty()) { - canContainDroppedPartitions = - ManifestFileUtil.canContainAny(manifest, dropPartitions, specsById); - } else { - canContainDroppedPartitions = false; + return ManifestFileUtil.canContainAny(manifest, dropPartitions, specsById); } - boolean canContainDroppedFiles; + return false; + } + + private boolean canContainDroppedFiles(ManifestFile manifest) { if (!deletePaths.isEmpty()) { - canContainDroppedFiles = true; + return true; } else if (!deleteFiles.isEmpty()) { - // because there were no path-only deletes, the set of deleted file partitions is valid - canContainDroppedFiles = - ManifestFileUtil.canContainAny(manifest, deleteFilePartitions, specsById); - } else { - canContainDroppedFiles = false; + return ManifestFileUtil.canContainAny(manifest, deleteFilePartitions, specsById); } - boolean canContainDropBySeq = - manifest.content() == ManifestContent.DELETES - && manifest.minSequenceNumber() < minSequenceNumber; - - return canContainExpressionDeletes - || canContainDroppedPartitions - || canContainDroppedFiles - || canContainDropBySeq; + return false; } @SuppressWarnings({"CollectionUndefinedEquality", "checkstyle:CyclomaticComplexity"}) private boolean manifestHasDeletedFiles( - PartitionAndMetricsEvaluator evaluator, ManifestReader reader) { + PartitionAndMetricsEvaluator evaluator, ManifestFile manifest, ManifestReader reader) { + if (manifestsWithDeletes.contains(manifest.path())) { + return true; + } + boolean isDelete = reader.isDeleteManifestReader(); for (ManifestEntry entry : reader.liveEntries()) { diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 0045a7486254..ee09357ab09d 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -579,7 +579,10 @@ public void testDeleteByExpressionWithDeleteFile() { @TestTemplate public void testDeleteDataFileWithDeleteFile() { - commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(fileADeletes()), branch); + commit( + table, + table.newRowDelta().addRows(FILE_A).addDeletes(fileADeletes()).addDeletes(fileBDeletes()), + branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); assertThat(latestSnapshot(table, branch).sequenceNumber()).isEqualTo(1); @@ -604,18 +607,18 @@ public void testDeleteDataFileWithDeleteFile() { assertThat(deleteSnap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( deleteSnap.deleteManifests(table.io()).get(0), - dataSeqs(1L), - fileSeqs(1L), - ids(deltaSnapshotId), - files(fileADeletes()), - statuses(Status.ADDED)); + dataSeqs(1L, 1L), + fileSeqs(1L, 1L), + ids(deltaSnapshotId, deltaSnapshotId), + files(fileADeletes(), fileBDeletes()), + statuses(Status.ADDED, Status.ADDED)); // the manifest that removed FILE_A will be dropped next commit, causing the min sequence number // of all data files // to be 2, the largest known sequence number. this will cause FILE_A_DELETES to be removed // because it is too old // to apply to any data files. - commit(table, table.newDelete().deleteFile("no-such-file"), branch); + commit(table, table.newRowDelta().removeDeletes(FILE_B_DELETES), branch); Snapshot nextSnap = latestSnapshot(table, branch); assertThat(nextSnap.sequenceNumber()).isEqualTo(3); @@ -625,11 +628,11 @@ public void testDeleteDataFileWithDeleteFile() { assertThat(nextSnap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( nextSnap.deleteManifests(table.io()).get(0), - dataSeqs(1L), - fileSeqs(1L), - ids(nextSnap.snapshotId()), - files(fileADeletes()), - statuses(Status.DELETED)); + dataSeqs(1L, 1L), + fileSeqs(1L, 1L), + ids(nextSnap.snapshotId(), nextSnap.snapshotId()), + files(fileADeletes(), fileBDeletes()), + statuses(Status.DELETED, Status.DELETED)); } @TestTemplate @@ -1444,6 +1447,59 @@ public void testRewrittenDeleteFiles() { statuses(Status.DELETED)); } + @TestTemplate + public void testRewrittenDeleteFilesReadFromManifest() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + DataFile dataFile = newDataFile("data_bucket=0"); + DeleteFile deleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta baseRowDelta = table.newRowDelta().addRows(dataFile).addDeletes(deleteFile); + Snapshot baseSnapshot = commit(table, baseRowDelta, branch); + assertThat(baseSnapshot.operation()).isEqualTo(DataOperations.OVERWRITE); + List deleteManifests = baseSnapshot.deleteManifests(table.io()); + try (ManifestReader deleteReader = + ManifestFiles.readDeleteManifest(deleteManifests.get(0), table.io(), table.specs())) { + deleteFile = deleteReader.iterator().next(); + } + + assertThat(deleteFile.manifestLocation()).isEqualTo(deleteManifests.get(0).path()); + DeleteFile newDeleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta rowDelta = + table + .newRowDelta() + .removeDeletes(deleteFile) + .addDeletes(newDeleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()); + Snapshot snapshot = commit(table, rowDelta, branch); + assertThat(snapshot.operation()).isEqualTo(DataOperations.DELETE); + + List dataManifests = snapshot.dataManifests(table.io()); + assertThat(dataManifests).hasSize(1); + validateManifest( + dataManifests.get(0), + dataSeqs(1L), + fileSeqs(1L), + ids(baseSnapshot.snapshotId()), + files(dataFile), + statuses(Status.ADDED)); + + deleteManifests = snapshot.deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(2); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(2L), + fileSeqs(2L), + ids(snapshot.snapshotId()), + files(newDeleteFile), + statuses(Status.ADDED)); + validateDeleteManifest( + deleteManifests.get(1), + dataSeqs(1L), + fileSeqs(1L), + ids(snapshot.snapshotId()), + files(deleteFile), + statuses(Status.DELETED)); + } + @TestTemplate public void testConcurrentDeletesRewriteSameDeleteFile() { assumeThat(formatVersion).isEqualTo(2); @@ -1509,6 +1565,70 @@ public void testConcurrentDeletesRewriteSameDeleteFile() { statuses(Status.ADDED)); } + @TestTemplate + public void testConcurrentManifestRewriteWithDeleteFileRemoval() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + // Manifest rewrite isn't supported on branches currently + assumeThat(branch).isEqualTo("main"); + + DataFile dataFile = newDataFile("data_bucket=0"); + DeleteFile deleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta rowDelta = table.newRowDelta().addRows(dataFile).addDeletes(deleteFile); + Snapshot first = commit(table, rowDelta, branch); + + DeleteFile secondDeleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); + Snapshot secondRowDelta = + commit(table, table.newRowDelta().addRows(dataFile).addDeletes(secondDeleteFile), branch); + List secondRowDeltaDeleteManifests = secondRowDelta.deleteManifests(table.io()); + assertThat(secondRowDeltaDeleteManifests).hasSize(2); + + // Read the manifest entries before the manifest rewrite is committed + List> readEntries = Lists.newArrayList(); + for (ManifestFile manifest : secondRowDeltaDeleteManifests) { + try (ManifestReader deleteManifestReader = + ManifestFiles.readDeleteManifest(manifest, table.io(), table.specs())) { + deleteManifestReader.entries().forEach(readEntries::add); + } + } + + RowDelta removeDeletes = + table + .newRowDelta() + .removeDeletes(readEntries.get(0).file()) + .removeDeletes(readEntries.get(1).file()) + .validateFromSnapshot(secondRowDelta.snapshotId()); + + RewriteManifests rewriteManifests = + table + .rewriteManifests() + .addManifest( + writeManifest( + "new_delete_manifest.avro", + // Specify data sequence number so that the delete files don't get aged out + // first + manifestEntry( + ManifestEntry.Status.EXISTING, first.snapshotId(), 3L, 0L, deleteFile), + manifestEntry( + ManifestEntry.Status.EXISTING, + secondRowDelta.snapshotId(), + 3L, + 0L, + secondDeleteFile))) + .deleteManifest(secondRowDeltaDeleteManifests.get(0)) + .deleteManifest(secondRowDeltaDeleteManifests.get(1)); + commit(table, rewriteManifests, branch); + + Snapshot remove = commit(table, removeDeletes, branch); + List deleteManifests = remove.deleteManifests(table.io()); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(3L, 3L), + fileSeqs(0L, 0L), + ids(remove.snapshotId(), remove.snapshotId()), + files(deleteFile, secondDeleteFile), + statuses(Status.DELETED, Status.DELETED)); + } + @TestTemplate public void testConcurrentMergeRewriteSameDeleteFile() { DataFile dataFile = newDataFile("data_bucket=0"); diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 2042906c15c6..363cbc52bcf2 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -28,11 +28,13 @@ import java.util.List; import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.TestTemplate; @@ -727,4 +729,130 @@ public void testCommitProperties() { .containsEntry( TableProperties.COMMIT_TOTAL_RETRY_TIME_MS, Integer.toString(60 * 60 * 1000)); } + + @TestTemplate + public void testRowDeltaWithConcurrentManifestRewrite() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + String branch = "main"; + RowDelta rowDelta = table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES); + Snapshot first = commit(table, rowDelta, branch); + + Snapshot secondRowDelta = + commit(table, table.newRowDelta().addRows(FILE_B).addDeletes(FILE_B_DELETES), branch); + List secondRowDeltaDeleteManifests = secondRowDelta.deleteManifests(table.io()); + assertThat(secondRowDeltaDeleteManifests).hasSize(2); + + // Read the manifest entries before the manifest rewrite is committed so that referenced + // manifests are populated + List> readEntries = Lists.newArrayList(); + for (ManifestFile manifest : secondRowDeltaDeleteManifests) { + try (ManifestReader deleteManifestReader = + ManifestFiles.readDeleteManifest(manifest, table.io(), table.specs())) { + deleteManifestReader.entries().forEach(readEntries::add); + } + } + + Transaction transaction = table.newTransaction(); + RowDelta removeDeletes = + transaction + .newRowDelta() + .removeDeletes(readEntries.get(0).file()) + .removeDeletes(readEntries.get(1).file()) + .validateFromSnapshot(secondRowDelta.snapshotId()); + removeDeletes.commit(); + + // cause the row delta transaction commit to fail and retry + RewriteManifests rewriteManifests = + table + .rewriteManifests() + .addManifest( + writeManifest( + "new_delete_manifest.avro", + // Specify data sequence number so that the delete files don't get aged out + // first + manifestEntry( + ManifestEntry.Status.EXISTING, first.snapshotId(), 3L, 0L, FILE_A_DELETES), + manifestEntry( + ManifestEntry.Status.EXISTING, + secondRowDelta.snapshotId(), + 3L, + 0L, + FILE_B_DELETES))) + .deleteManifest(secondRowDeltaDeleteManifests.get(0)) + .deleteManifest(secondRowDeltaDeleteManifests.get(1)); + commit(table, rewriteManifests, branch); + + transaction.commitTransaction(); + Snapshot removedDeletes = table.currentSnapshot(); + List deleteManifests = removedDeletes.deleteManifests(table.io()); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(3L, 3L), + fileSeqs(0L, 0L), + ids(removedDeletes.snapshotId(), removedDeletes.snapshotId()), + files(FILE_A_DELETES, FILE_B_DELETES), + statuses(Status.DELETED, Status.DELETED)); + } + + @TestTemplate + public void testOverwriteWithConcurrentManifestRewrite() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + String branch = "main"; + OverwriteFiles overwrite = table.newOverwrite().addFile(FILE_A).addFile(FILE_A2); + Snapshot first = commit(table, overwrite, branch); + + overwrite = table.newOverwrite().addFile(FILE_B); + Snapshot second = commit(table, overwrite, branch); + List secondOverwriteManifests = second.dataManifests(table.io()); + assertThat(secondOverwriteManifests).hasSize(2); + + // Read the manifest entries before the manifest rewrite is committed so that referenced + // manifests are populated + List> entries = Lists.newArrayList(); + for (ManifestFile manifest : secondOverwriteManifests) { + try (ManifestReader manifestReader = + ManifestFiles.read(manifest, table.io(), table.specs())) { + manifestReader.entries().forEach(entries::add); + } + } + + ManifestEntry removedDataFileEntry = + entries.stream() + .filter(entry -> entry.file().location().equals(FILE_A2.location())) + .collect(Collectors.toList()) + .get(0); + + Transaction overwriteTransaction = table.newTransaction(); + OverwriteFiles overwriteFiles = + overwriteTransaction + .newOverwrite() + .deleteFile(removedDataFileEntry.file()) + .validateFromSnapshot(second.snapshotId()); + overwriteFiles.commit(); + + // cause the overwrite transaction commit to fail and retry + RewriteManifests rewriteManifests = + table + .rewriteManifests() + .addManifest( + writeManifest( + "new_manifest.avro", + manifestEntry(Status.EXISTING, first.snapshotId(), FILE_A), + manifestEntry(Status.EXISTING, first.snapshotId(), FILE_A2), + manifestEntry(Status.EXISTING, second.snapshotId(), FILE_B))) + .deleteManifest(secondOverwriteManifests.get(0)) + .deleteManifest(secondOverwriteManifests.get(1)); + commit(table, rewriteManifests, branch); + + overwriteTransaction.commitTransaction(); + Snapshot latestOverwrite = table.currentSnapshot(); + List manifests = latestOverwrite.dataManifests(table.io()); + validateManifest( + manifests.get(0), + dataSeqs(0L, 0L, 0L), + fileSeqs(0L, 0L, 0L), + ids(first.snapshotId(), latestOverwrite.snapshotId(), second.snapshotId()), + files(FILE_A, FILE_A2, FILE_B), + statuses(Status.EXISTING, Status.DELETED, Status.EXISTING)); + } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 25dc7e47e971..9c3c7e589c07 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.PartitionStatisticsFile; @@ -298,8 +299,8 @@ public void testExpireDeleteFiles() throws Exception { "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); - Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); + DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); + Path deleteFilePath = new Path(String.valueOf(deleteFile.path())); sql( "CALL %s.system.rewrite_data_files(" @@ -310,9 +311,10 @@ public void testExpireDeleteFiles() throws Exception { catalogName, tableIdent); table.refresh(); - sql( - "INSERT INTO TABLE %s VALUES (5, 'e')", - tableName); // this txn moves the file to the DELETED state + table + .newRowDelta() + .removeDeletes(deleteFile) + .commit(); // this txn moves the file to the DELETED state sql("INSERT INTO TABLE %s VALUES (6, 'f')", tableName); // this txn removes the file reference table.refresh(); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 8235aca7f764..8c355612ceb5 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.PartitionStatisticsFile; @@ -290,8 +291,8 @@ public void testExpireDeleteFiles() throws Exception { "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); - Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); + DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); + Path deleteFilePath = new Path(String.valueOf(deleteFile.path())); sql( "CALL %s.system.rewrite_data_files(" @@ -302,9 +303,10 @@ public void testExpireDeleteFiles() throws Exception { catalogName, tableIdent); table.refresh(); - sql( - "INSERT INTO TABLE %s VALUES (5, 'e')", - tableName); // this txn moves the file to the DELETED state + table + .newRowDelta() + .removeDeletes(deleteFile) + .commit(); // this txn moves the file to the DELETED state sql("INSERT INTO TABLE %s VALUES (6, 'f')", tableName); // this txn removes the file reference table.refresh(); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 34fec09add7c..c58277ec084d 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.PartitionStatisticsFile; @@ -282,8 +283,8 @@ public void testExpireDeleteFiles() throws Exception { assertThat(TestHelpers.deleteManifests(table)).as("Should have 1 delete manifest").hasSize(1); assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); - Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().location())); + DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); + Path deleteFilePath = new Path(String.valueOf(deleteFile.location())); sql( "CALL %s.system.rewrite_data_files(" @@ -294,9 +295,10 @@ public void testExpireDeleteFiles() throws Exception { catalogName, tableIdent); table.refresh(); - sql( - "INSERT INTO TABLE %s VALUES (5, 'e')", - tableName); // this txn moves the file to the DELETED state + table + .newRowDelta() + .removeDeletes(deleteFile) + .commit(); // this txn moves the file to the DELETED state sql("INSERT INTO TABLE %s VALUES (6, 'f')", tableName); // this txn removes the file reference table.refresh(); From 12845d4edc0dcb65c7049509bbd54dc16396c6d1 Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Thu, 21 Nov 2024 21:09:10 +0100 Subject: [PATCH 176/313] Revert "Core: Update TableMetadataParser to ensure all streams closed (#11220)" (#11621) This reverts commit 2b55fef7cc2a249d864ac26d85a4923313d96a59. --- .../java/org/apache/iceberg/TableMetadataParser.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index c244b3996c9e..d7f2b29be75a 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -122,9 +122,9 @@ public static void write(TableMetadata metadata, OutputFile outputFile) { public static void internalWrite( TableMetadata metadata, OutputFile outputFile, boolean overwrite) { boolean isGzip = Codec.fromFileName(outputFile.location()) == Codec.GZIP; - try (OutputStream os = overwrite ? outputFile.createOrOverwrite() : outputFile.create(); - OutputStream gos = isGzip ? new GZIPOutputStream(os) : os; - OutputStreamWriter writer = new OutputStreamWriter(gos, StandardCharsets.UTF_8)) { + OutputStream stream = overwrite ? outputFile.createOrOverwrite() : outputFile.create(); + try (OutputStream ou = isGzip ? new GZIPOutputStream(stream) : stream; + OutputStreamWriter writer = new OutputStreamWriter(ou, StandardCharsets.UTF_8)) { JsonGenerator generator = JsonUtil.factory().createGenerator(writer); generator.useDefaultPrettyPrinter(); toJson(metadata, generator); @@ -277,9 +277,9 @@ public static TableMetadata read(FileIO io, String path) { public static TableMetadata read(FileIO io, InputFile file) { Codec codec = Codec.fromFileName(file.location()); - try (InputStream is = file.newStream(); - InputStream gis = codec == Codec.GZIP ? new GZIPInputStream(is) : is) { - return fromJson(file, JsonUtil.mapper().readValue(gis, JsonNode.class)); + try (InputStream is = + codec == Codec.GZIP ? new GZIPInputStream(file.newStream()) : file.newStream()) { + return fromJson(file, JsonUtil.mapper().readValue(is, JsonNode.class)); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to read file: %s", file); } From a52afdc484f9e63477b96d0182586692daac8eab Mon Sep 17 00:00:00 2001 From: Haizhou Zhao Date: Thu, 21 Nov 2024 15:32:26 -0800 Subject: [PATCH 177/313] Add REST Catalog tests to Spark 3.5 integration test (#11093) * Add REST Catalog tests to Spark 3.5 integration test Add REST Catalog tests to Spark 3.4 integration test tmp save Fix integ tests Revert "Add REST Catalog tests to Spark 3.4 integration test" This reverts commit d052416d4e8a2c26ece3d250e1b213a41a7f7cdd. unneeded changes fix test retrigger checks Fix integ test Fix port already in use Fix unmatched validation catalog spotless Fix sqlite related test failures * Rebase & spotless * code format * unneeded change * unneeded change * Revert "unneeded change" This reverts commit ae29c41769aefa0b2ae6f6102c4646337af5cad3. * code format * Use in-mem config to configure RCK * Update open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java * Use RESTServerExtension * check style and test failure * test failure * fix test * fix test * spotless * Update open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java Co-authored-by: Eduard Tudenhoefner * Update open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java Co-authored-by: Eduard Tudenhoefner * Update spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java Co-authored-by: Eduard Tudenhoefner * Spotless and fix test * Apply suggestions from code review * Apply suggestions from code review * Apply suggestions from code review * Update spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java * Package protected RCKUtils * spotless * unintentional change * remove warehouse specification from rest * spotless * move find free port to rest server extension * fix typo * checkstyle * fix unit test --------- Co-authored-by: Haizhou Zhao Co-authored-by: Eduard Tudenhoefner --- .../org/apache/iceberg/rest/RCKUtils.java | 22 ++++++- .../iceberg/rest/RESTCatalogServer.java | 15 ++++- .../iceberg/rest/RESTServerExtension.java | 35 +++++++++- spark/v3.5/build.gradle | 22 +++++++ .../spark/extensions/TestMetadataTables.java | 10 ++- .../TestRemoveOrphanFilesProcedure.java | 12 ++-- .../apache/iceberg/spark/CatalogTestBase.java | 10 +++ .../iceberg/spark/SparkCatalogConfig.java | 4 ++ .../iceberg/spark/TestBaseWithCatalog.java | 65 +++++++++++++++++-- .../actions/TestComputeTableStatsAction.java | 1 + .../iceberg/spark/sql/TestAlterTable.java | 6 ++ .../iceberg/spark/sql/TestCreateTable.java | 6 ++ .../iceberg/spark/sql/TestRefreshTable.java | 8 ++- 13 files changed, 195 insertions(+), 21 deletions(-) diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java index 0f1571d362e3..adeba4709329 100644 --- a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.rest; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.ServerSocket; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -77,14 +80,21 @@ static Map environmentCatalogConfig() { } static RESTCatalog initCatalogClient() { + return initCatalogClient(Maps.newHashMap()); + } + + static RESTCatalog initCatalogClient(Map properties) { Map catalogProperties = Maps.newHashMap(); catalogProperties.putAll(RCKUtils.environmentCatalogConfig()); catalogProperties.putAll(Maps.fromProperties(System.getProperties())); + catalogProperties.putAll(properties); // Set defaults + String port = + catalogProperties.getOrDefault( + RESTCatalogServer.REST_PORT, String.valueOf(RESTCatalogServer.REST_PORT_DEFAULT)); catalogProperties.putIfAbsent( - CatalogProperties.URI, - String.format("http://localhost:%s/", RESTCatalogServer.REST_PORT_DEFAULT)); + CatalogProperties.URI, String.format("http://localhost:%s/", port)); catalogProperties.putIfAbsent(CatalogProperties.WAREHOUSE_LOCATION, "rck_warehouse"); RESTCatalog catalog = new RESTCatalog(); @@ -107,4 +117,12 @@ static void purgeCatalogTestEntries(RESTCatalog catalog) { catalog.dropNamespace(namespace); }); } + + static int findFreePort() { + try (ServerSocket socket = new ServerSocket(0)) { + return socket.getLocalPort(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } } diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java index b3d12f74e4b0..e79a590127fd 100644 --- a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java @@ -26,6 +26,7 @@ import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.gzip.GzipHandler; @@ -37,12 +38,19 @@ public class RESTCatalogServer { private static final Logger LOG = LoggerFactory.getLogger(RESTCatalogServer.class); - static final String REST_PORT = "rest.port"; + public static final String REST_PORT = "rest.port"; static final int REST_PORT_DEFAULT = 8181; private Server httpServer; + private final Map config; - RESTCatalogServer() {} + RESTCatalogServer() { + this.config = Maps.newHashMap(); + } + + RESTCatalogServer(Map config) { + this.config = config; + } static class CatalogContext { private final Catalog catalog; @@ -64,7 +72,8 @@ public Map configuration() { private CatalogContext initializeBackendCatalog() throws IOException { // Translate environment variables to catalog properties - Map catalogProperties = RCKUtils.environmentCatalogConfig(); + Map catalogProperties = Maps.newHashMap(RCKUtils.environmentCatalogConfig()); + catalogProperties.putAll(config); // Fallback to a JDBCCatalog impl if one is not set catalogProperties.putIfAbsent(CatalogProperties.CATALOG_IMPL, JdbcCatalog.class.getName()); diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java index fc3648055694..19236bec64c3 100644 --- a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java @@ -18,19 +18,49 @@ */ package org.apache.iceberg.rest; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; import org.junit.jupiter.api.extension.ExtensionContext; public class RESTServerExtension implements BeforeAllCallback, AfterAllCallback { + // if the caller explicitly wants the server to start on port 0, it means the caller wants to + // launch on a free port + public static final String FREE_PORT = "0"; + private RESTCatalogServer localServer; + private RESTCatalog client; + private final Map config; + + public RESTServerExtension() { + config = Maps.newHashMap(); + } + + public RESTServerExtension(Map config) { + Map conf = Maps.newHashMap(config); + if (conf.containsKey(RESTCatalogServer.REST_PORT) + && conf.get(RESTCatalogServer.REST_PORT).equals(FREE_PORT)) { + conf.put(RESTCatalogServer.REST_PORT, String.valueOf(RCKUtils.findFreePort())); + } + this.config = conf; + } + + public Map config() { + return config; + } + + public RESTCatalog client() { + return client; + } @Override public void beforeAll(ExtensionContext extensionContext) throws Exception { if (Boolean.parseBoolean( extensionContext.getConfigurationParameter(RCKUtils.RCK_LOCAL).orElse("true"))) { - this.localServer = new RESTCatalogServer(); + this.localServer = new RESTCatalogServer(config); this.localServer.start(false); + this.client = RCKUtils.initCatalogClient(config); } } @@ -39,5 +69,8 @@ public void afterAll(ExtensionContext extensionContext) throws Exception { if (localServer != null) { localServer.stop(); } + if (client != null) { + client.close(); + } } } diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index e3c9ef4f0230..cfb5912a774f 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -107,8 +107,13 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') + testImplementation (project(path: ':iceberg-open-api', configuration: 'testFixturesRuntimeElements')) { + transitive = false + } testImplementation libs.sqlite.jdbc testImplementation libs.awaitility + // runtime dependencies for running REST Catalog based integration test + testRuntimeOnly libs.jetty.servlet } test { @@ -172,6 +177,12 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') testImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + testImplementation (project(path: ':iceberg-open-api', configuration: 'testFixturesRuntimeElements')) { + transitive = false + } + // runtime dependencies for running REST Catalog based integration test + testRuntimeOnly libs.jetty.servlet + testRuntimeOnly libs.sqlite.jdbc testImplementation libs.avro.avro testImplementation libs.parquet.hadoop @@ -255,6 +266,17 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') integrationImplementation project(path: ":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + + // runtime dependencies for running Hive Catalog based integration test + integrationRuntimeOnly project(':iceberg-hive-metastore') + // runtime dependencies for running REST Catalog based integration test + integrationRuntimeOnly project(path: ':iceberg-core', configuration: 'testArtifacts') + integrationRuntimeOnly (project(path: ':iceberg-open-api', configuration: 'testFixturesRuntimeElements')) { + transitive = false + } + integrationRuntimeOnly libs.jetty.servlet + integrationRuntimeOnly libs.sqlite.jdbc + // Not allowed on our classpath, only the runtime jar is allowed integrationCompileOnly project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}") integrationCompileOnly project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index a22cf61ec8c9..cd623a9d6a42 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.spark.extensions; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; import java.util.Comparator; @@ -521,7 +524,7 @@ public void testFilesTableTimeTravelWithSchemaEvolution() throws Exception { optional(3, "category", Types.StringType.get()))); spark.createDataFrame(newRecords, newSparkSchema).coalesce(1).writeTo(tableName).append(); - + table.refresh(); Long currentSnapshotId = table.currentSnapshot().snapshotId(); Dataset actualFilesDs = @@ -740,6 +743,11 @@ private boolean partitionMatch(Record file, String partValue) { @TestTemplate public void metadataLogEntriesAfterReplacingTable() throws Exception { + assumeThat(catalogConfig.get(ICEBERG_CATALOG_TYPE)) + .as( + "need to fix https://github.com/apache/iceberg/issues/11109 before enabling this for the REST catalog") + .isNotEqualTo(ICEBERG_CATALOG_TYPE_REST); + sql( "CREATE TABLE %s (id bigint, data string) " + "USING iceberg " diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 0928cde7ca66..e7698e5d5d07 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -450,12 +450,14 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { Table table = Spark3Util.loadIcebergTable(spark, tableName); String statsFileName = "stats-file-" + UUID.randomUUID(); + String location = table.location(); + // not every catalog will return file proto for local directories + // i.e. Hadoop and Hive Catalog do, Jdbc and REST do not + if (!location.startsWith("file:")) { + location = "file:" + location; + } File statsLocation = - new File(new URI(table.location())) - .toPath() - .resolve("data") - .resolve(statsFileName) - .toFile(); + new File(new URI(location)).toPath().resolve("data").resolve(statsFileName).toFile(); StatisticsFile statisticsFile; try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) { long snapshotId = table.currentSnapshot().snapshotId(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java index ba864bf89e33..6cc100097c7a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java @@ -18,8 +18,10 @@ */ package org.apache.iceberg.spark; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.extension.ExtendWith; @ExtendWith(ParameterizedTestExtension.class) @@ -43,6 +45,14 @@ protected static Object[][] parameters() { SparkCatalogConfig.SPARK.catalogName(), SparkCatalogConfig.SPARK.implementation(), SparkCatalogConfig.SPARK.properties() + }, + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + .build() } }; } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java index abfd7da0c7bd..e03f500dc46c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java @@ -34,6 +34,10 @@ public enum SparkCatalogConfig { "testhadoop", SparkCatalog.class.getName(), ImmutableMap.of("type", "hadoop", "cache-enabled", "false")), + REST( + "testrest", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "rest", "cache-enabled", "false")), SPARK( "spark_catalog", SparkSessionCatalog.class.getName(), diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java index c869c4a30a19..96dbd94a805d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java @@ -18,6 +18,11 @@ */ package org.apache.iceberg.spark; +import static org.apache.iceberg.CatalogProperties.CATALOG_IMPL; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; @@ -36,17 +41,38 @@ import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.rest.RESTCatalogServer; +import org.apache.iceberg.rest.RESTServerExtension; import org.apache.iceberg.util.PropertyUtil; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; @ExtendWith(ParameterizedTestExtension.class) public abstract class TestBaseWithCatalog extends TestBase { protected static File warehouse = null; + @RegisterExtension + private static final RESTServerExtension REST_SERVER_EXTENSION = + new RESTServerExtension( + Map.of( + RESTCatalogServer.REST_PORT, + RESTServerExtension.FREE_PORT, + // In-memory sqlite database by default is private to the connection that created it. + // If more than 1 jdbc connection backed by in-memory sqlite is created behind one + // JdbcCatalog, then different jdbc connections could provide different views of table + // status even belonging to the same catalog. Reference: + // https://www.sqlite.org/inmemorydb.html + CatalogProperties.CLIENT_POOL_SIZE, + "1")); + + protected static RESTCatalog restCatalog; + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") protected static Object[][] parameters() { return new Object[][] { @@ -59,13 +85,14 @@ protected static Object[][] parameters() { } @BeforeAll - public static void createWarehouse() throws IOException { + public static void setUpAll() throws IOException { TestBaseWithCatalog.warehouse = File.createTempFile("warehouse", null); assertThat(warehouse.delete()).isTrue(); + restCatalog = REST_SERVER_EXTENSION.client(); } @AfterAll - public static void dropWarehouse() throws IOException { + public static void tearDownAll() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); @@ -89,13 +116,37 @@ public static void dropWarehouse() throws IOException { protected TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table"); protected String tableName; + private void configureValidationCatalog() { + if (catalogConfig.containsKey(ICEBERG_CATALOG_TYPE)) { + switch (catalogConfig.get(ICEBERG_CATALOG_TYPE)) { + case ICEBERG_CATALOG_TYPE_HADOOP: + this.validationCatalog = + new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse); + break; + case ICEBERG_CATALOG_TYPE_REST: + this.validationCatalog = restCatalog; + break; + case ICEBERG_CATALOG_TYPE_HIVE: + this.validationCatalog = catalog; + break; + default: + throw new IllegalArgumentException("Unknown catalog type"); + } + } else if (catalogConfig.containsKey(CATALOG_IMPL)) { + switch (catalogConfig.get(CATALOG_IMPL)) { + case "org.apache.iceberg.inmemory.InMemoryCatalog": + this.validationCatalog = new InMemoryCatalog(); + break; + default: + throw new IllegalArgumentException("Unknown catalog impl"); + } + } + this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + } + @BeforeEach public void before() { - this.validationCatalog = - catalogName.equals("testhadoop") - ? new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse) - : catalog; - this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + configureValidationCatalog(); spark.conf().set("spark.sql.catalog." + catalogName, implementation); catalogConfig.forEach( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java index 7aa849d0bba8..057ef231ca1d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java @@ -110,6 +110,7 @@ public void testComputeTableStatsAction() throws NoSuchTableException, ParseExce new SimpleRecord(4, "d")); spark.createDataset(records, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); SparkActions actions = SparkActions.get(); + table.refresh(); ComputeTableStats.Result results = actions.computeTableStats(table).columns("id", "data").execute(); assertThat(results).isNotNull(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java index 7c98888f1667..5abc72606f9f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; @@ -275,6 +277,10 @@ public void testAlterColumnPositionFirst() { @TestTemplate public void testTableRename() { + assumeThat(catalogConfig.get(ICEBERG_CATALOG_TYPE)) + .as( + "need to fix https://github.com/apache/iceberg/issues/11154 before enabling this for the REST catalog") + .isNotEqualTo(ICEBERG_CATALOG_TYPE_REST); assumeThat(validationCatalog) .as("Hadoop catalog does not support rename") .isNotInstanceOf(HadoopCatalog.class); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index 11d4cfebfea6..39aeacf68b9a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; @@ -351,6 +353,10 @@ public void testCreateTableProperties() { @TestTemplate public void testCreateTableCommitProperties() { + assumeThat(catalogConfig.get(ICEBERG_CATALOG_TYPE)) + .as( + "need to fix https://github.com/apache/iceberg/issues/11554 before enabling this for the REST catalog") + .isNotEqualTo(ICEBERG_CATALOG_TYPE_REST); assertThat(validationCatalog.tableExists(tableIdent)) .as("Table should not already exist") .isFalse(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java index 8a9ae0f6030a..fe13d61db066 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.sql; import java.util.List; +import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -45,8 +46,11 @@ public void removeTables() { public void testRefreshCommand() { // We are not allowed to change the session catalog after it has been initialized, so build a // new one - if (catalogName.equals(SparkCatalogConfig.SPARK.catalogName()) - || catalogName.equals(SparkCatalogConfig.HADOOP.catalogName())) { + if (Set.of( + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.REST.catalogName()) + .contains(catalogName)) { spark.conf().set("spark.sql.catalog." + catalogName + ".cache-enabled", true); spark = spark.cloneSession(); } From ce4c44792a31fdadb99489bca50360d9d190877d Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 22 Nov 2024 22:26:23 +0800 Subject: [PATCH 178/313] Spark 3.5: Correct the two-stage parsing strategy of antlr parser (#11628) --- .../extensions/IcebergSparkSqlExtensionsParser.scala | 8 ++++++-- .../apache/iceberg/spark/extensions/TestBranchDDL.java | 6 +++--- .../org/apache/iceberg/spark/extensions/TestTagDDL.java | 8 ++++---- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index 30940d9cffc2..72023a6f2f41 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -181,19 +181,23 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI parser.removeErrorListeners() parser.addErrorListener(IcebergParseErrorListener) + // https://github.com/antlr/antlr4/issues/192#issuecomment-15238595 + // Save a great deal of time on correct inputs by using a two-stage parsing strategy. try { try { - // first, try parsing with potentially faster SLL mode + // first, try parsing with potentially faster SLL mode and BailErrorStrategy + parser.setErrorHandler(new BailErrorStrategy) parser.getInterpreter.setPredictionMode(PredictionMode.SLL) toResult(parser) } catch { case _: ParseCancellationException => - // if we fail, parse with LL mode + // if we fail, parse with LL mode with DefaultErrorStrategy tokenStream.seek(0) // rewind input stream parser.reset() // Try Again. + parser.setErrorHandler(new DefaultErrorStrategy) parser.getInterpreter.setPredictionMode(PredictionMode.LL) toResult(parser) } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index fb7f73186ad3..6bad39dc5818 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -218,13 +218,13 @@ public void testCreateBranchUseCustomMaxRefAge() throws NoSuchTableException { assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input"); assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input"); assertThatThrownBy( () -> @@ -274,7 +274,7 @@ public void testDropBranchFailsForTag() throws NoSuchTableException { public void testDropBranchNonConformingName() { assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input '123'"); + .hasMessageContaining("no viable alternative at input '123'"); } @TestTemplate diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java index 65c2c0f713cb..623af8777475 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java @@ -99,12 +99,12 @@ public void testCreateTagWithRetain() throws NoSuchTableException { "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN", tableName, tagName, firstSnapshotId, maxRefAge)) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input ''"); assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE TAG %s RETAIN %s DAYS", tableName, tagName, "abc")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input 'abc'"); assertThatThrownBy( () -> @@ -151,7 +151,7 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input '123'"); + .hasMessageContaining("no viable alternative at input '123'"); table.manageSnapshots().removeTag(tagName).commit(); List records = @@ -303,7 +303,7 @@ public void testDropTag() throws NoSuchTableException { public void testDropTagNonConformingName() { assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input '123'"); + .hasMessageContaining("no viable alternative at input '123'"); } @TestTemplate From f717ebde46b235681165636da0c8f44bfa1823e5 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 22 Nov 2024 22:26:47 +0800 Subject: [PATCH 179/313] Spark 3.4: Correct the two-stage parsing strategy of antlr parser (#7734) --- .../extensions/IcebergSparkSqlExtensionsParser.scala | 8 ++++++-- .../apache/iceberg/spark/extensions/TestBranchDDL.java | 6 +++--- .../org/apache/iceberg/spark/extensions/TestTagDDL.java | 8 ++++---- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index f29e31efba41..e45dd2d65ac9 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -238,19 +238,23 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI parser.removeErrorListeners() parser.addErrorListener(IcebergParseErrorListener) + // https://github.com/antlr/antlr4/issues/192#issuecomment-15238595 + // Save a great deal of time on correct inputs by using a two-stage parsing strategy. try { try { - // first, try parsing with potentially faster SLL mode + // first, try parsing with potentially faster SLL mode and BailErrorStrategy + parser.setErrorHandler(new BailErrorStrategy) parser.getInterpreter.setPredictionMode(PredictionMode.SLL) toResult(parser) } catch { case _: ParseCancellationException => - // if we fail, parse with LL mode + // if we fail, parse with LL mode with DefaultErrorStrategy tokenStream.seek(0) // rewind input stream parser.reset() // Try Again. + parser.setErrorHandler(new DefaultErrorStrategy) parser.getInterpreter.setPredictionMode(PredictionMode.LL) toResult(parser) } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 6302c7d3f45b..69f328fc66a6 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -217,13 +217,13 @@ public void testCreateBranchUseCustomMaxRefAge() throws NoSuchTableException { assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input"); assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input"); assertThatThrownBy( () -> @@ -273,7 +273,7 @@ public void testDropBranchFailsForTag() throws NoSuchTableException { public void testDropBranchNonConformingName() { assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input '123'"); + .hasMessageContaining("no viable alternative at input '123'"); } @Test diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java index b8673d0ae3bc..e70326c39eae 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java @@ -102,12 +102,12 @@ public void testCreateTagWithRetain() throws NoSuchTableException { "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN", tableName, tagName, firstSnapshotId, maxRefAge)) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input ''"); assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE TAG %s RETAIN %s DAYS", tableName, tagName, "abc")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input 'abc'"); assertThatThrownBy( () -> @@ -152,7 +152,7 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input '123'"); + .hasMessageContaining("no viable alternative at input '123'"); table.manageSnapshots().removeTag(tagName).commit(); List records = @@ -306,7 +306,7 @@ public void testDropTag() throws NoSuchTableException { public void testDropTagNonConformingName() { assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input '123'"); + .hasMessageContaining("no viable alternative at input '123'"); } @Test From f2b1b91d304039027333570451477e7575f7d39d Mon Sep 17 00:00:00 2001 From: ismail simsek <6005685+ismailsimsek@users.noreply.github.com> Date: Fri, 22 Nov 2024 15:49:27 +0100 Subject: [PATCH 180/313] Docs: Add new blog post to Iceberg Blogs (#11627) --- site/docs/blogs.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 0d29ebcec56c..1cf0b745f6b3 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -22,6 +22,12 @@ title: "Blogs" Here is a list of company blogs that talk about Iceberg. The blogs are ordered from most recent to oldest. + +### [Building a Data Lake with Debezium and Apache Iceberg](https://medium.com/@ismail-simsek/building-a-data-lake-with-debezium-and-apache-iceberg-part-1-25124daf2a95) +**Date**: November 15th, 2024, **Company**: Memiiso Community + +**Author**: [Ismail Simsek](https://www.linkedin.com/in/ismailsimsek/) + ### [Hands-on with Apache Iceberg Tables using PyIceberg using Nessie and Minio](https://www.dremio.com/blog/intro-to-pyiceberg/) **Date**: October 22nd, 2024, **Company**: Dremio From 5851ca6e0797bb84d669f27bb9ebc4a7b2a00361 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 23 Nov 2024 14:28:24 +0800 Subject: [PATCH 181/313] Docs: Mention HIVE-28121 for MySQL/MariaDB-based HMS users (#11631) --- docs/docs/configuration.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index 07a98fd94515..100c4d35771e 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -186,6 +186,8 @@ This should only be set to `false` if all following conditions are met: - [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882) is available on the Hive Metastore server + - [HIVE-28121](https://issues.apache.org/jira/browse/HIVE-28121) +is available on the Hive Metastore server, if it is backed by MySQL or MariaDB - All other HiveCatalogs committing to tables that this HiveCatalog commits to are also on Iceberg 1.3 or later - All other HiveCatalogs committing to tables that this HiveCatalog commits to have also disabled Hive locks on commit. From 9cc13b11fbcb0091d6ad94882a1019dddf6b9023 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 23 Nov 2024 14:57:50 +0800 Subject: [PATCH 182/313] Spark 3.4: IcebergSource extends SessionConfigSupport (#7732) --- .../iceberg/spark/source/IcebergSource.java | 9 +++- .../source/TestIcebergSourceTablesBase.java | 46 +++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java index 8975c7f32db1..d084c06b2b7c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java @@ -38,6 +38,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager; import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.SessionConfigSupport; import org.apache.spark.sql.connector.catalog.SupportsCatalogOptions; import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -61,7 +62,8 @@ *

    The above list is in order of priority. For example: a matching catalog will take priority * over any namespace resolution. */ -public class IcebergSource implements DataSourceRegister, SupportsCatalogOptions { +public class IcebergSource + implements DataSourceRegister, SupportsCatalogOptions, SessionConfigSupport { private static final String DEFAULT_CATALOG_NAME = "default_iceberg"; private static final String DEFAULT_CACHE_CATALOG_NAME = "default_cache_iceberg"; private static final String DEFAULT_CATALOG = "spark.sql.catalog." + DEFAULT_CATALOG_NAME; @@ -80,6 +82,11 @@ public String shortName() { return "iceberg"; } + @Override + public String keyPrefix() { + return shortName(); + } + @Override public StructType inferSchema(CaseInsensitiveStringMap options) { return null; diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index f56f1161e624..84dff6666423 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -2184,6 +2184,52 @@ private void testWithFilter(String filterExpr, TableIdentifier tableIdentifier) assertThat(actual).as("Rows must match").containsExactlyInAnyOrderElementsOf(expected); } + @Test + public void testSessionConfigSupport() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + TableIdentifier tableIdentifier = TableIdentifier.of("db", "session_config_table"); + Table table = createTable(tableIdentifier, SCHEMA, spec); + + List initialRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(initialRecords, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + long s1 = table.currentSnapshot().snapshotId(); + + withSQLConf( + // set write option through session configuration + ImmutableMap.of("spark.datasource.iceberg.snapshot-property.foo", "bar"), + () -> { + df.select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + }); + + table.refresh(); + assertThat(table.currentSnapshot().summary()).containsEntry("foo", "bar"); + + withSQLConf( + // set read option through session configuration + ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), + () -> { + Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual) + .as("Rows must match") + .containsExactlyInAnyOrderElementsOf(initialRecords); + }); + } + private GenericData.Record manifestRecord( Table manifestTable, Long referenceSnapshotId, ManifestFile manifest) { GenericRecordBuilder builder = From 5e09cdc6b908aa8cbfa191739cc1ad1e6db652ce Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 23 Nov 2024 14:58:39 +0800 Subject: [PATCH 183/313] Spark 3.5: IcebergSource extends SessionConfigSupport (#11624) --- .../iceberg/spark/source/IcebergSource.java | 9 +++- .../source/TestIcebergSourceTablesBase.java | 46 +++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java index 8975c7f32db1..d084c06b2b7c 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java @@ -38,6 +38,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager; import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.SessionConfigSupport; import org.apache.spark.sql.connector.catalog.SupportsCatalogOptions; import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -61,7 +62,8 @@ *

    The above list is in order of priority. For example: a matching catalog will take priority * over any namespace resolution. */ -public class IcebergSource implements DataSourceRegister, SupportsCatalogOptions { +public class IcebergSource + implements DataSourceRegister, SupportsCatalogOptions, SessionConfigSupport { private static final String DEFAULT_CATALOG_NAME = "default_iceberg"; private static final String DEFAULT_CACHE_CATALOG_NAME = "default_cache_iceberg"; private static final String DEFAULT_CATALOG = "spark.sql.catalog." + DEFAULT_CATALOG_NAME; @@ -80,6 +82,11 @@ public String shortName() { return "iceberg"; } + @Override + public String keyPrefix() { + return shortName(); + } + @Override public StructType inferSchema(CaseInsensitiveStringMap options) { return null; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 29216150d362..21afd7460ec6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -2210,6 +2210,52 @@ private void testWithFilter(String filterExpr, TableIdentifier tableIdentifier) assertThat(actual).as("Rows must match").containsExactlyInAnyOrderElementsOf(expected); } + @Test + public void testSessionConfigSupport() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + TableIdentifier tableIdentifier = TableIdentifier.of("db", "session_config_table"); + Table table = createTable(tableIdentifier, SCHEMA, spec); + + List initialRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(initialRecords, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + long s1 = table.currentSnapshot().snapshotId(); + + withSQLConf( + // set write option through session configuration + ImmutableMap.of("spark.datasource.iceberg.snapshot-property.foo", "bar"), + () -> { + df.select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + }); + + table.refresh(); + assertThat(table.currentSnapshot().summary()).containsEntry("foo", "bar"); + + withSQLConf( + // set read option through session configuration + ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), + () -> { + Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual) + .as("Rows must match") + .containsExactlyInAnyOrderElementsOf(initialRecords); + }); + } + private GenericData.Record manifestRecord( Table manifestTable, Long referenceSnapshotId, ManifestFile manifest) { GenericRecordBuilder builder = From eddf9a161c47def40bd1104cda41a38c45dada44 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 23 Nov 2024 14:59:46 +0800 Subject: [PATCH 184/313] Spark 3.3: IcebergSource extends SessionConfigSupport (#11625) --- .../iceberg/spark/source/IcebergSource.java | 9 +++- .../source/TestIcebergSourceTablesBase.java | 46 +++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java index 8975c7f32db1..d084c06b2b7c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java @@ -38,6 +38,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager; import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.SessionConfigSupport; import org.apache.spark.sql.connector.catalog.SupportsCatalogOptions; import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -61,7 +62,8 @@ *

    The above list is in order of priority. For example: a matching catalog will take priority * over any namespace resolution. */ -public class IcebergSource implements DataSourceRegister, SupportsCatalogOptions { +public class IcebergSource + implements DataSourceRegister, SupportsCatalogOptions, SessionConfigSupport { private static final String DEFAULT_CATALOG_NAME = "default_iceberg"; private static final String DEFAULT_CACHE_CATALOG_NAME = "default_cache_iceberg"; private static final String DEFAULT_CATALOG = "spark.sql.catalog." + DEFAULT_CATALOG_NAME; @@ -80,6 +82,11 @@ public String shortName() { return "iceberg"; } + @Override + public String keyPrefix() { + return shortName(); + } + @Override public StructType inferSchema(CaseInsensitiveStringMap options) { return null; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 12011b66a5be..d82bc0b4ba8e 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -2185,6 +2185,52 @@ private void testWithFilter(String filterExpr, TableIdentifier tableIdentifier) assertThat(actual).as("Rows must match").containsExactlyInAnyOrderElementsOf(expected); } + @Test + public void testSessionConfigSupport() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + TableIdentifier tableIdentifier = TableIdentifier.of("db", "session_config_table"); + Table table = createTable(tableIdentifier, SCHEMA, spec); + + List initialRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(initialRecords, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + long s1 = table.currentSnapshot().snapshotId(); + + withSQLConf( + // set write option through session configuration + ImmutableMap.of("spark.datasource.iceberg.snapshot-property.foo", "bar"), + () -> { + df.select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + }); + + table.refresh(); + assertThat(table.currentSnapshot().summary()).containsEntry("foo", "bar"); + + withSQLConf( + // set read option through session configuration + ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), + () -> { + Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual) + .as("Rows must match") + .containsExactlyInAnyOrderElementsOf(initialRecords); + }); + } + private GenericData.Record manifestRecord( Table manifestTable, Long referenceSnapshotId, ManifestFile manifest) { GenericRecordBuilder builder = From b1fbef7bcb57969fd7de6dd421eb6a72b5416182 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 25 Nov 2024 08:43:23 +0100 Subject: [PATCH 185/313] Build: Bump testcontainers from 1.20.3 to 1.20.4 (#11640) Bumps `testcontainers` from 1.20.3 to 1.20.4. Updates `org.testcontainers:testcontainers` from 1.20.3 to 1.20.4 - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/1.20.3...1.20.4) Updates `org.testcontainers:junit-jupiter` from 1.20.3 to 1.20.4 - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/1.20.3...1.20.4) Updates `org.testcontainers:minio` from 1.20.3 to 1.20.4 - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/1.20.3...1.20.4) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.testcontainers:junit-jupiter dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.testcontainers:minio dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 7358d0b8547a..f3db1b53b2db 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -84,7 +84,7 @@ spark-hive33 = "3.3.4" spark-hive34 = "3.4.4" spark-hive35 = "3.5.2" sqlite-jdbc = "3.47.0.0" -testcontainers = "1.20.3" +testcontainers = "1.20.4" tez010 = "0.10.4" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 3aebcfeb8a12af11ca868968db206c4fdb7cce4d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 25 Nov 2024 08:45:53 +0100 Subject: [PATCH 186/313] Build: Bump mkdocs-material from 9.5.44 to 9.5.45 (#11641) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.44 to 9.5.45. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.5.44...9.5.45) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 28056f56c84b..7cbbaba724ca 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.3.7 -mkdocs-material==9.5.44 +mkdocs-material==9.5.45 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.2 From 4337040be07565608b945300ceabca91659c7766 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 25 Nov 2024 15:48:17 +0800 Subject: [PATCH 187/313] Spark 3.3: Correct the two-stage parsing strategy of antlr parser (#11630) --- .../extensions/IcebergSparkSqlExtensionsParser.scala | 8 ++++++-- .../apache/iceberg/spark/extensions/TestBranchDDL.java | 6 +++--- .../org/apache/iceberg/spark/extensions/TestTagDDL.java | 8 ++++---- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index b281cfedb7bc..f0324697e5a1 100644 --- a/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -238,19 +238,23 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI parser.removeErrorListeners() parser.addErrorListener(IcebergParseErrorListener) + // https://github.com/antlr/antlr4/issues/192#issuecomment-15238595 + // Save a great deal of time on correct inputs by using a two-stage parsing strategy. try { try { - // first, try parsing with potentially faster SLL mode + // first, try parsing with potentially faster SLL mode and BailErrorStrategy + parser.setErrorHandler(new BailErrorStrategy) parser.getInterpreter.setPredictionMode(PredictionMode.SLL) toResult(parser) } catch { case _: ParseCancellationException => - // if we fail, parse with LL mode + // if we fail, parse with LL mode with DefaultErrorStrategy tokenStream.seek(0) // rewind input stream parser.reset() // Try Again. + parser.setErrorHandler(new DefaultErrorStrategy) parser.getInterpreter.setPredictionMode(PredictionMode.LL) toResult(parser) } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java index 762ade946583..7911053ab117 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -220,14 +220,14 @@ public void testCreateBranchUseCustomMaxRefAge() throws NoSuchTableException { assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) .as("Illegal statement") .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input"); assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")) .as("Illegal statement") .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input"); assertThatThrownBy( () -> @@ -281,7 +281,7 @@ public void testDropBranchNonConformingName() { assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) .as("Non-conforming branch name") .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input '123'"); + .hasMessageContaining("no viable alternative at input '123'"); } @Test diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java index 6c1418b7a796..ada62b8f9ba5 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java @@ -103,13 +103,13 @@ public void testCreateTagWithRetain() throws NoSuchTableException { tableName, tagName, firstSnapshotId, maxRefAge)) .as("Illegal statement") .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input ''"); assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE TAG %s RETAIN %s DAYS", tableName, tagName, "abc")) .as("Illegal statement") .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input"); + .hasMessageContaining("no viable alternative at input 'abc'"); assertThatThrownBy( () -> @@ -158,7 +158,7 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) .as("Non-conforming tag name") .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input '123'"); + .hasMessageContaining("no viable alternative at input '123'"); table.manageSnapshots().removeTag(tagName).commit(); List records = @@ -315,7 +315,7 @@ public void testDropTagNonConformingName() { assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) .as("Non-conforming tag name") .isInstanceOf(IcebergParseException.class) - .hasMessageContaining("mismatched input '123'"); + .hasMessageContaining("no viable alternative at input '123'"); } @Test From 1f23dcd0e1c1a2360e2741f0ef562ca7fdd05eab Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 25 Nov 2024 08:49:38 +0100 Subject: [PATCH 188/313] Build: Bump software.amazon.awssdk:bom from 2.29.15 to 2.29.20 (#11639) Bumps software.amazon.awssdk:bom from 2.29.15 to 2.29.20. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index f3db1b53b2db..989e7a76b966 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.29.15" +awssdk-bom = "2.29.20" azuresdk-bom = "1.2.29" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" From 4b52dbd89bf583f1bbdbf0b7a73b62404812ee11 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 25 Nov 2024 10:25:56 +0100 Subject: [PATCH 189/313] Core,Open-API: Don't expose the `last-column-id` (#11514) * Core,Open-API: Don't expose the `last-column-id` Okay, I've added this to the spec a while ago: https://github.com/apache/iceberg/pull/7445 But I think this was a mistake, and we should not expose this to the public APIs, as it is much better to track this internally. I noticed this while reviewing https://github.com/apache/iceberg-rust/pull/587 Removing this as part of the APIs in Java, and the Open-API update makes it much more resilient, and don't require the clients to compute this value. For example. when there are two conflicting schema changes, the last-column-id must be recomputed correctly when doing the retry operation. * Update the tests as well * Add `deprecation` flag * Wording Co-authored-by: Eduard Tudenhoefner * Wording Co-authored-by: Eduard Tudenhoefner * Wording * Thanks Ryan! * Remove `LOG` --------- Co-authored-by: Eduard Tudenhoefner --- .../aws/glue/TestIcebergToGlueConverter.java | 2 +- .../org/apache/iceberg/MetadataUpdate.java | 10 ++++++++ .../java/org/apache/iceberg/SchemaUpdate.java | 2 +- .../org/apache/iceberg/TableMetadata.java | 25 ++++++++++++++++++- .../iceberg/rest/RESTSessionCatalog.java | 2 +- .../org/apache/iceberg/view/ViewMetadata.java | 7 +----- .../iceberg/TestMetadataUpdateParser.java | 18 ++----------- .../org/apache/iceberg/TestTableMetadata.java | 16 ++++++------ .../iceberg/TestUpdateRequirements.java | 18 ++++++------- open-api/rest-catalog-open-api.py | 2 +- open-api/rest-catalog-open-api.yaml | 8 +++++- .../source/TestSparkMetadataColumns.java | 5 +--- .../spark/source/TestSparkReadProjection.java | 3 +-- 13 files changed, 67 insertions(+), 51 deletions(-) diff --git a/aws/src/test/java/org/apache/iceberg/aws/glue/TestIcebergToGlueConverter.java b/aws/src/test/java/org/apache/iceberg/aws/glue/TestIcebergToGlueConverter.java index 1136ad63b410..edebfd3420e2 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/glue/TestIcebergToGlueConverter.java +++ b/aws/src/test/java/org/apache/iceberg/aws/glue/TestIcebergToGlueConverter.java @@ -238,7 +238,7 @@ public void testSetTableInputInformationWithRemovedColumns() { Schema newSchema = new Schema(Types.NestedField.required(1, "x", Types.StringType.get(), "comment1")); - tableMetadata = tableMetadata.updateSchema(newSchema, 3); + tableMetadata = tableMetadata.updateSchema(newSchema); IcebergToGlueConverter.setTableInputInformation(actualTableInputBuilder, tableMetadata); TableInput actualTableInput = actualTableInputBuilder.build(); diff --git a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java index 49fb1fe01c44..ba038c196e43 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java +++ b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java @@ -86,6 +86,16 @@ class AddSchema implements MetadataUpdate { private final Schema schema; private final int lastColumnId; + public AddSchema(Schema schema) { + this(schema, schema.highestFieldId()); + } + + /** + * Set the schema + * + * @deprecated since 1.8.0, will be removed 1.9.0 or 2.0.0, use AddSchema(schema). + */ + @Deprecated public AddSchema(Schema schema, int lastColumnId) { this.schema = schema; this.lastColumnId = lastColumnId; diff --git a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java index 069097778606..2b541080ac72 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java +++ b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java @@ -444,7 +444,7 @@ public Schema apply() { @Override public void commit() { - TableMetadata update = applyChangesToMetadata(base.updateSchema(apply(), lastColumnId)); + TableMetadata update = applyChangesToMetadata(base.updateSchema(apply())); ops.commit(base, update); } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 0e323bca1c97..9f6ffbcc8714 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -563,10 +563,23 @@ public TableMetadata withUUID() { return new Builder(this).assignUUID().build(); } + /** + * Updates the schema + * + * @deprecated since 1.8.0, will be removed in 1.9.0 or 2.0.0, use updateSchema(schema). + */ + @Deprecated public TableMetadata updateSchema(Schema newSchema, int newLastColumnId) { return new Builder(this).setCurrentSchema(newSchema, newLastColumnId).build(); } + /** Updates the schema */ + public TableMetadata updateSchema(Schema newSchema) { + return new Builder(this) + .setCurrentSchema(newSchema, Math.max(this.lastColumnId, newSchema.highestFieldId())) + .build(); + } + // The caller is responsible to pass a newPartitionSpec with correct partition field IDs public TableMetadata updatePartitionSpec(PartitionSpec newPartitionSpec) { return new Builder(this).setDefaultPartitionSpec(newPartitionSpec).build(); @@ -1082,8 +1095,18 @@ public Builder setCurrentSchema(int schemaId) { return this; } + public Builder addSchema(Schema schema) { + addSchemaInternal(schema, Math.max(lastColumnId, schema.highestFieldId())); + return this; + } + + /** + * Add a new schema. + * + * @deprecated since 1.8.0, will be removed in 1.9.0 or 2.0.0, use AddSchema(schema). + */ + @Deprecated public Builder addSchema(Schema schema, int newLastColumnId) { - // TODO: remove requirement for newLastColumnId addSchemaInternal(schema, newLastColumnId); return this; } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index b89595635314..1bf57dd13c69 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -954,7 +954,7 @@ private static List createChanges(TableMetadata meta) { changes.add(new MetadataUpdate.UpgradeFormatVersion(meta.formatVersion())); Schema schema = meta.schema(); - changes.add(new MetadataUpdate.AddSchema(schema, schema.highestFieldId())); + changes.add(new MetadataUpdate.AddSchema(schema)); changes.add(new MetadataUpdate.SetCurrentSchema(-1)); PartitionSpec spec = meta.spec(); diff --git a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java index ae837ff96882..94f3a56ba931 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java @@ -372,20 +372,15 @@ private int addSchemaInternal(Schema schema) { newSchema = schema; } - int highestFieldId = Math.max(highestFieldId(), newSchema.highestFieldId()); schemas.add(newSchema); schemasById.put(newSchema.schemaId(), newSchema); - changes.add(new MetadataUpdate.AddSchema(newSchema, highestFieldId)); + changes.add(new MetadataUpdate.AddSchema(newSchema)); this.lastAddedSchemaId = newSchemaId; return newSchemaId; } - private int highestFieldId() { - return schemas.stream().map(Schema::highestFieldId).max(Integer::compareTo).orElse(0); - } - private int reuseOrCreateNewSchemaId(Schema newSchema) { // if the schema already exists, use its id; otherwise use the highest id + 1 int newSchemaId = INITIAL_SCHEMA_ID; diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java index bfed6ebebe2c..cae19fece4e9 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java @@ -112,23 +112,9 @@ public void testUpgradeFormatVersionFromJson() { public void testAddSchemaFromJson() { String action = MetadataUpdateParser.ADD_SCHEMA; Schema schema = ID_DATA_SCHEMA; - int lastColumnId = schema.highestFieldId(); - String json = - String.format( - "{\"action\":\"add-schema\",\"schema\":%s,\"last-column-id\":%d}", - SchemaParser.toJson(schema), lastColumnId); - MetadataUpdate actualUpdate = new MetadataUpdate.AddSchema(schema, lastColumnId); - assertEquals(action, actualUpdate, MetadataUpdateParser.fromJson(json)); - } - - @Test - public void testAddSchemaFromJsonWithoutLastColumnId() { - String action = MetadataUpdateParser.ADD_SCHEMA; - Schema schema = ID_DATA_SCHEMA; - int lastColumnId = schema.highestFieldId(); String json = String.format("{\"action\":\"add-schema\",\"schema\":%s}", SchemaParser.toJson(schema)); - MetadataUpdate actualUpdate = new MetadataUpdate.AddSchema(schema, lastColumnId); + MetadataUpdate actualUpdate = new MetadataUpdate.AddSchema(schema); assertEquals(action, actualUpdate, MetadataUpdateParser.fromJson(json)); } @@ -140,7 +126,7 @@ public void testAddSchemaToJson() { String.format( "{\"action\":\"add-schema\",\"schema\":%s,\"last-column-id\":%d}", SchemaParser.toJson(schema), lastColumnId); - MetadataUpdate update = new MetadataUpdate.AddSchema(schema, lastColumnId); + MetadataUpdate update = new MetadataUpdate.AddSchema(schema); String actual = MetadataUpdateParser.toJson(update); assertThat(actual) .as("Add schema should convert to the correct JSON value") diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index c9a8eb75a986..64c410b46427 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -1425,7 +1425,7 @@ public void testUpdateSchemaIdentifierFields() { new Schema( Lists.newArrayList(Types.NestedField.required(1, "x", Types.StringType.get())), Sets.newHashSet(1)); - TableMetadata newMeta = meta.updateSchema(newSchema, 1); + TableMetadata newMeta = meta.updateSchema(newSchema); assertThat(newMeta.schemas()).hasSize(2); assertThat(newMeta.schema().identifierFieldIds()).containsExactly(1); } @@ -1447,7 +1447,7 @@ public void testUpdateSchema() { new Schema( Types.NestedField.required(1, "y", Types.LongType.get(), "comment"), Types.NestedField.required(2, "x", Types.StringType.get())); - TableMetadata twoSchemasTable = freshTable.updateSchema(schema2, 2); + TableMetadata twoSchemasTable = freshTable.updateSchema(schema2); assertThat(twoSchemasTable.currentSchemaId()).isEqualTo(1); assertSameSchemaList( ImmutableList.of(schema, new Schema(1, schema2.columns())), twoSchemasTable.schemas()); @@ -1459,26 +1459,26 @@ public void testUpdateSchema() { new Schema( Types.NestedField.required(1, "y", Types.LongType.get(), "comment"), Types.NestedField.required(2, "x", Types.StringType.get())); - TableMetadata sameSchemaTable = twoSchemasTable.updateSchema(sameSchema2, 2); + TableMetadata sameSchemaTable = twoSchemasTable.updateSchema(sameSchema2); assertThat(sameSchemaTable).isSameAs(twoSchemasTable); // update schema with the same schema and different last column ID as current should create // a new table - TableMetadata differentColumnIdTable = sameSchemaTable.updateSchema(sameSchema2, 3); + TableMetadata differentColumnIdTable = sameSchemaTable.updateSchema(sameSchema2); assertThat(differentColumnIdTable.currentSchemaId()).isEqualTo(1); assertSameSchemaList( ImmutableList.of(schema, new Schema(1, schema2.columns())), differentColumnIdTable.schemas()); assertThat(differentColumnIdTable.schema().asStruct()).isEqualTo(schema2.asStruct()); - assertThat(differentColumnIdTable.lastColumnId()).isEqualTo(3); + assertThat(differentColumnIdTable.lastColumnId()).isEqualTo(2); // update schema with old schema does not change schemas - TableMetadata revertSchemaTable = differentColumnIdTable.updateSchema(schema, 3); + TableMetadata revertSchemaTable = differentColumnIdTable.updateSchema(schema); assertThat(revertSchemaTable.currentSchemaId()).isEqualTo(0); assertSameSchemaList( ImmutableList.of(schema, new Schema(1, schema2.columns())), revertSchemaTable.schemas()); assertThat(revertSchemaTable.schema().asStruct()).isEqualTo(schema.asStruct()); - assertThat(revertSchemaTable.lastColumnId()).isEqualTo(3); + assertThat(revertSchemaTable.lastColumnId()).isEqualTo(2); // create new schema will use the largest schema id + 1 Schema schema3 = @@ -1486,7 +1486,7 @@ public void testUpdateSchema() { Types.NestedField.required(2, "y", Types.LongType.get(), "comment"), Types.NestedField.required(4, "x", Types.StringType.get()), Types.NestedField.required(6, "z", Types.IntegerType.get())); - TableMetadata threeSchemaTable = revertSchemaTable.updateSchema(schema3, 6); + TableMetadata threeSchemaTable = revertSchemaTable.updateSchema(schema3); assertThat(threeSchemaTable.currentSchemaId()).isEqualTo(2); assertSameSchemaList( ImmutableList.of( diff --git a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java index 1a6c289ea241..e5b342850820 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java @@ -223,9 +223,9 @@ public void addSchema() { UpdateRequirements.forUpdateTable( metadata, ImmutableList.of( - new MetadataUpdate.AddSchema(new Schema(), lastColumnId), - new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 1), - new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 2))); + new MetadataUpdate.AddSchema(new Schema()), + new MetadataUpdate.AddSchema(new Schema()), + new MetadataUpdate.AddSchema(new Schema()))); requirements.forEach(req -> req.validate(metadata)); assertThat(requirements) @@ -253,9 +253,9 @@ public void addSchemaFailure() { UpdateRequirements.forUpdateTable( metadata, ImmutableList.of( - new MetadataUpdate.AddSchema(new Schema(), 1), - new MetadataUpdate.AddSchema(new Schema(), 2), - new MetadataUpdate.AddSchema(new Schema(), 3))); + new MetadataUpdate.AddSchema(new Schema()), + new MetadataUpdate.AddSchema(new Schema()), + new MetadataUpdate.AddSchema(new Schema()))); assertThatThrownBy(() -> requirements.forEach(req -> req.validate(updated))) .isInstanceOf(CommitFailedException.class) @@ -269,9 +269,9 @@ public void addSchemaForView() { UpdateRequirements.forReplaceView( viewMetadata, ImmutableList.of( - new MetadataUpdate.AddSchema(new Schema(), lastColumnId), - new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 1), - new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 2))); + new MetadataUpdate.AddSchema(new Schema()), + new MetadataUpdate.AddSchema(new Schema()), + new MetadataUpdate.AddSchema(new Schema()))); requirements.forEach(req -> req.validate(viewMetadata)); assertThat(requirements) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index c3372544ef95..d63e9bfe548c 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -1152,7 +1152,7 @@ class AddSchemaUpdate(BaseUpdate): last_column_id: Optional[int] = Field( None, alias='last-column-id', - description='The highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side.', + description="This optional field is **DEPRECATED for REMOVAL** since it more safe to handle this internally, and shouldn't be exposed to the clients.\nThe highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side.", ) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 9635af96c1ca..a154ce97b564 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2692,7 +2692,13 @@ components: $ref: '#/components/schemas/Schema' last-column-id: type: integer - description: The highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side. + deprecated: true + description: + This optional field is **DEPRECATED for REMOVAL** since it more safe to handle this internally, + and shouldn't be exposed to the clients. + + The highest assigned column ID for the table. This is used to ensure columns are always + assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side. SetCurrentSchemaUpdate: allOf: diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index 230a660c0117..93f39299116b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -189,10 +189,7 @@ public void testPartitionMetadataColumnWithManyColumns() { TableOperations ops = ((HasTableOperations) table).operations(); TableMetadata base = ops.current(); - ops.commit( - base, - base.updateSchema(manyColumnsSchema, manyColumnsSchema.highestFieldId()) - .updatePartitionSpec(spec)); + ops.commit(base, base.updateSchema(manyColumnsSchema).updatePartitionSpec(spec)); Dataset df = spark diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java index 99a327402d97..becf6a064dcc 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java @@ -152,8 +152,7 @@ protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema Schema expectedSchema = reassignIds(readSchema, idMapping); // Set the schema to the expected schema directly to simulate the table schema evolving - TestTables.replaceMetadata( - desc, TestTables.readMetadata(desc).updateSchema(expectedSchema, 100)); + TestTables.replaceMetadata(desc, TestTables.readMetadata(desc).updateSchema(expectedSchema)); Dataset df = spark From cb1ad79cab49a3bd7202835b68b29596e37c0f37 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 25 Nov 2024 10:28:40 +0100 Subject: [PATCH 190/313] Build: Bump com.google.errorprone:error_prone_annotations (#11638) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.35.1 to 2.36.0. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.35.1...v2.36.0) --- updated-dependencies: - dependency-name: com.google.errorprone:error_prone_annotations dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 989e7a76b966..c8bd64a8fb29 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -38,7 +38,7 @@ datasketches = "6.1.1" delta-standalone = "3.2.1" delta-spark = "3.2.1" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.35.1" +errorprone-annotations = "2.36.0" failsafe = "3.3.2" findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} From fa47f3141f8c163c4ddcbf1a02e9af85f3f5df04 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Mon, 25 Nov 2024 13:23:10 +0100 Subject: [PATCH 191/313] Flink: Add table.exec.iceberg.use-v2-sink option (#11244) --- docs/docs/flink-writes.md | 25 ++++++ .../iceberg/flink/FlinkConfigOptions.java | 6 ++ .../iceberg/flink/IcebergTableSink.java | 28 +++++-- .../apache/iceberg/flink/sink/FlinkSink.java | 12 ++- .../iceberg/flink/TestFlinkTableSink.java | 28 ++++++- .../flink/TestFlinkTableSinkExtended.java | 82 +++++++++++++++---- 6 files changed, 148 insertions(+), 33 deletions(-) diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index f53b5d832efe..3edd2720a1f4 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -371,3 +371,28 @@ and [deleting orphan files](maintenance.md#delete-orphan-files) could possibly c the state of the Flink job. To avoid that, make sure to keep the last snapshot created by the Flink job (which can be identified by the `flink.job-id` property in the summary), and only delete orphan files that are old enough. + +# Flink Writes (SinkV2 based implementation) + +At the time when the current default, `FlinkSink` implementation was created, Flink Sink's interface had some +limitations that were not acceptable for the Iceberg tables purpose. Due to these limitations, `FlinkSink` is based +on a custom chain of `StreamOperator`s terminated by `DiscardingSink`. + +In the 1.15 version of Flink [SinkV2 interface](https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction) +was introduced. This interface is used in the new `IcebergSink` implementation which is available in the `iceberg-flink` module. +The new implementation is a base for further work on features such as [table maintenance](maintenance.md). +The SinkV2 based implementation is currently an experimental feature so use it with caution. + +## Writing with SQL + +To turn on SinkV2 based implementation in SQL, set this configuration option: +```sql +SET table.exec.iceberg.use-v2-sink = true; +``` + +## Writing with DataStream + +To use SinkV2 based implementation, replace `FlinkSink` with `IcebergSink` in the provided snippets. +Warning: There are some slight differences between these implementations: +- The `RANGE` distribution mode is not yet available for the `IcebergSink` +- When using `IcebergSink` use `uidSuffix` instead of the `uidPrefix` diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java index 099925101571..97e2c70d348e 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java @@ -91,6 +91,12 @@ private FlinkConfigOptions() {} .defaultValue(true) .withDescription("Use the FLIP-27 based Iceberg source implementation."); + public static final ConfigOption TABLE_EXEC_ICEBERG_USE_V2_SINK = + ConfigOptions.key("table.exec.iceberg.use-v2-sink") + .booleanType() + .defaultValue(false) + .withDescription("Use the SinkV2 API based Iceberg sink implementation."); + public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = ConfigOptions.key("table.exec.iceberg.split-assigner-type") .enumType(SplitAssignerType.class) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java index 1b9268569d9a..c2c5a6706e92 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java @@ -35,6 +35,7 @@ import org.apache.flink.types.RowKind; import org.apache.flink.util.Preconditions; import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.flink.sink.IcebergSink; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; public class IcebergTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite { @@ -77,14 +78,25 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { @Override public DataStreamSink consumeDataStream( ProviderContext providerContext, DataStream dataStream) { - return FlinkSink.forRowData(dataStream) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .equalityFieldColumns(equalityColumns) - .overwrite(overwrite) - .setAll(writeProps) - .flinkConf(readableConfig) - .append(); + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK)) { + return IcebergSink.forRowData(dataStream) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .equalityFieldColumns(equalityColumns) + .overwrite(overwrite) + .setAll(writeProps) + .flinkConf(readableConfig) + .append(); + } else { + return FlinkSink.forRowData(dataStream) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .equalityFieldColumns(equalityColumns) + .overwrite(overwrite) + .setAll(writeProps) + .flinkConf(readableConfig) + .append(); + } } }; } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index e862e88c968c..2e586b960c22 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -41,7 +41,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; @@ -392,7 +392,7 @@ public Builder toBranch(String branch) { return this; } - private DataStreamSink chainIcebergOperators() { + private DataStreamSink chainIcebergOperators() { Preconditions.checkArgument( inputCreator != null, "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); @@ -484,12 +484,10 @@ List checkAndGetEqualityFieldIds() { return equalityFieldIds; } - @SuppressWarnings("unchecked") - private DataStreamSink appendDummySink( - SingleOutputStreamOperator committerStream) { - DataStreamSink resultStream = + private DataStreamSink appendDummySink(SingleOutputStreamOperator committerStream) { + DataStreamSink resultStream = committerStream - .addSink(new DiscardingSink()) + .sinkTo(new DiscardingSink<>()) .name(operatorName(String.format("IcebergSink %s", this.table.name()))) .setParallelism(1); if (uidPrefix != null) { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index 2978a92945a2..fad30f9c1e67 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -40,7 +40,6 @@ public class TestFlinkTableSink extends CatalogTestBase { - private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; private static final String TABLE_NAME = "test_table"; private TableEnvironment tEnv; private Table icebergTable; @@ -51,7 +50,11 @@ public class TestFlinkTableSink extends CatalogTestBase { @Parameter(index = 3) private boolean isStreamingJob; - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") + @Parameter(index = 4) + private boolean useV2Sink; + + @Parameters( + name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}, useV2Sink={4}") public static List parameters() { List parameters = Lists.newArrayList(); for (FileFormat format : @@ -60,10 +63,24 @@ public static List parameters() { for (Object[] catalogParams : CatalogTestBase.parameters()) { String catalogName = (String) catalogParams[0]; Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); + parameters.add( + new Object[] { + catalogName, baseNamespace, format, isStreaming, false /* don't use v2 sink */ + }); } } } + + for (FileFormat format : + new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { + for (Boolean isStreaming : new Boolean[] {true, false}) { + String catalogName = "testhadoop_basenamespace"; + Namespace baseNamespace = Namespace.of("l0", "l1"); + parameters.add( + new Object[] {catalogName, baseNamespace, format, isStreaming, true /* use v2 sink */}); + } + } + return parameters; } @@ -87,6 +104,11 @@ protected TableEnvironment getTableEnv() { } } } + + tEnv.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK, useV2Sink); + return tEnv; } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java index b63547d433a4..3afabf6e0795 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -31,6 +31,8 @@ import java.util.stream.IntStream; import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.apache.flink.streaming.api.transformations.SinkTransformation; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; @@ -52,6 +54,7 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.sink.IcebergSink; import org.apache.iceberg.flink.source.BoundedTableFactory; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -91,11 +94,20 @@ public class TestFlinkTableSinkExtended extends SqlBase { private TableEnvironment tEnv; - @Parameter protected boolean isStreamingJob; + @Parameter(index = 0) + protected boolean isStreamingJob; - @Parameters(name = "isStreamingJob={0}") + @Parameter(index = 1) + protected Boolean useV2Sink; + + @Parameters(name = "isStreamingJob={0}, useV2Sink={1}") protected static List parameters() { - return Arrays.asList(new Boolean[] {true}, new Boolean[] {false}); + return Arrays.asList( + new Object[] {true, false}, + new Object[] {false, false}, + new Object[] {true, true}, + new Object[] {false, true}, + new Object[] {true, null}); } protected synchronized TableEnvironment getTableEnv() { @@ -115,6 +127,13 @@ protected synchronized TableEnvironment getTableEnv() { tEnv = TableEnvironment.create(settingsBuilder.build()); } } + + if (useV2Sink != null) { + tEnv.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK, useV2Sink); + } + return tEnv; } @@ -146,6 +165,32 @@ public void clean() throws Exception { catalog.close(); } + @TestTemplate + public void testUsedFlinkSinkInterface() { + String dataId = BoundedTableFactory.registerDataSet(Collections.emptyList()); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); + String insertSQL = String.format("INSERT INTO %s SELECT * FROM %s", TABLE, SOURCE_TABLE); + ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); + Transformation transformation = + planner.translate(Collections.singletonList(operation)).get(0); + assertThat(transformation).as("Should use SinkV2 API").isInstanceOf(SinkTransformation.class); + SinkTransformation sinkTransformation = (SinkTransformation) transformation; + if (useV2Sink != null && useV2Sink) { + assertThat(sinkTransformation.getSink()) + .as("Should use SinkV2 API based implementation") + .isInstanceOf(IcebergSink.class); + } else { + assertThat(sinkTransformation.getSink()) + .as("Should use custom chain of StreamOperators terminated by DiscardingSink") + .isInstanceOf(DiscardingSink.class); + } + } + @TestTemplate public void testWriteParallelism() { List dataSet = @@ -165,18 +210,25 @@ public void testWriteParallelism() { "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", TABLE, SOURCE_TABLE); ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); - Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); - Transformation committer = dummySink.getInputs().get(0); - Transformation writer = committer.getInputs().get(0); - - assertThat(writer.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); - writer - .getInputs() - .forEach( - input -> - assertThat(input.getParallelism()) - .as("Should have the expected parallelism.") - .isEqualTo(isStreamingJob ? 2 : 4)); + Transformation sink = planner.translate(Collections.singletonList(operation)).get(0); + if (useV2Sink != null && useV2Sink) { + assertThat(sink.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); + Transformation writerInput = sink.getInputs().get(0); + assertThat(writerInput.getParallelism()) + .as("Should have the expected parallelism.") + .isEqualTo(isStreamingJob ? 2 : 4); + } else { + Transformation committer = sink.getInputs().get(0); + Transformation writer = committer.getInputs().get(0); + + assertThat(writer.getParallelism()) + .as("Should have the expected 1 parallelism.") + .isEqualTo(1); + Transformation writerInput = writer.getInputs().get(0); + assertThat(writerInput.getParallelism()) + .as("Should have the expected parallelism.") + .isEqualTo(isStreamingJob ? 2 : 4); + } } @TestTemplate From cdd944ebbd42cd94103f15c8baad8daa82995846 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 25 Nov 2024 20:45:27 +0800 Subject: [PATCH 192/313] Docs: Use DataFrameWriterV2 in example (#11647) --- docs/docs/spark-configuration.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 5b281b19891a..8342d1d57727 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -167,14 +167,14 @@ spark.read ### Write options -Spark write options are passed when configuring the DataFrameWriter, like this: +Spark write options are passed when configuring the DataFrameWriterV2, like this: ```scala // write with Avro instead of Parquet -df.write +df.writeTo("catalog.db.table") .option("write-format", "avro") .option("snapshot-property.key", "value") - .insertInto("catalog.db.table") + .append() ``` | Spark option | Default | Description | From f7ff0dc8c0a27e2bcd727e4f7705cf0a69ccc9b3 Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Mon, 25 Nov 2024 14:08:54 +0100 Subject: [PATCH 193/313] Docs: Add `WHEN NOT MATCHED BY SOURCE` to Spark doc (#11636) --- docs/docs/spark-writes.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/docs/spark-writes.md b/docs/docs/spark-writes.md index cc8ca76fe5f7..3006de600457 100644 --- a/docs/docs/spark-writes.md +++ b/docs/docs/spark-writes.md @@ -95,6 +95,11 @@ WHEN NOT MATCHED AND s.event_time > still_valid_threshold THEN INSERT (id, count Only one record in the source data can update any given row of the target table, or else an error will be thrown. +Spark 3.5 added support for `WHEN NOT MATCHED BY SOURCE ... THEN ...` to update or delete rows that are not present in the source data: + +```sql +WHEN NOT MATCHED BY SOURCE THEN UPDATE SET status = 'invalid' +``` ### `INSERT OVERWRITE` From fa00482d3fb45a053d9569f82983295a5c6a7499 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 25 Nov 2024 22:25:31 +0100 Subject: [PATCH 194/313] Build: Bump nessie from 0.100.0 to 0.100.2 (#11637) Bumps `nessie` from 0.100.0 to 0.100.2. Updates `org.projectnessie.nessie:nessie-client` from 0.100.0 to 0.100.2 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.100.0 to 0.100.2 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.100.0 to 0.100.2 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.100.0 to 0.100.2 --- updated-dependencies: - dependency-name: org.projectnessie.nessie:nessie-client dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-jaxrs-testextension dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-testextension dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index c8bd64a8fb29..860e0654ea8b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -70,7 +70,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.100.0" +nessie = "0.100.2" netty-buffer = "4.1.115.Final" netty-buffer-compat = "4.1.115.Final" object-client-bundle = "3.3.2" From 430ebff8ea4f947a8cef1ad181b19ed5157ef26b Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 26 Nov 2024 15:37:46 +0800 Subject: [PATCH 195/313] Build: Delete branch automatically on PR merge (#11635) --- .asf.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.asf.yaml b/.asf.yaml index bf45b54a1787..d93e94ebf65f 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -39,6 +39,8 @@ github: required_approving_review_count: 1 required_linear_history: true + + del_branch_on_merge: true features: wiki: true From f3560871564c95a0e7b2bff3ca6ecb2e08726d01 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 26 Nov 2024 16:55:09 +0100 Subject: [PATCH 196/313] Flink: Test both "new" Flink Avro planned reader and "deprecated" Avro reader (#11430) --- ...=> AbstractTestFlinkAvroReaderWriter.java} | 11 ++-- .../TestFlinkAvroDeprecatedReaderWriter.java | 38 ++++++++++++ .../TestFlinkAvroPlannedReaderWriter.java | 34 +++++++++++ .../iceberg/flink/data/TestRowProjection.java | 58 +++++++++++++------ 4 files changed, 116 insertions(+), 25 deletions(-) rename flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/{TestFlinkAvroReaderWriter.java => AbstractTestFlinkAvroReaderWriter.java} (96%) create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java similarity index 96% rename from flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java index 2b9e8694b6d6..cbf49ae6faa9 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java @@ -48,7 +48,7 @@ import org.apache.iceberg.util.DateTimeUtil; import org.junit.jupiter.api.Test; -public class TestFlinkAvroReaderWriter extends DataTest { +public abstract class AbstractTestFlinkAvroReaderWriter extends DataTest { private static final int NUM_RECORDS = 100; @@ -70,6 +70,8 @@ protected void writeAndValidate(Schema schema) throws IOException { writeAndValidate(schema, expectedRecords, NUM_RECORDS); } + protected abstract Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema); + private void writeAndValidate(Schema schema, List expectedRecords, int numRecord) throws IOException { RowType flinkSchema = FlinkSchemaUtil.convert(schema); @@ -88,11 +90,7 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n writer.addAll(expectedRecords); } - try (CloseableIterable reader = - Avro.read(Files.localInput(recordsFile)) - .project(schema) - .createResolvingReader(FlinkPlannedAvroReader::create) - .build()) { + try (CloseableIterable reader = createAvroReadBuilder(recordsFile, schema).build()) { Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < numRecord; i++) { @@ -156,7 +154,6 @@ private Record recordNumType( @Test public void testNumericTypes() throws IOException { - List expected = ImmutableList.of( recordNumType( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java new file mode 100644 index 000000000000..03910f4fda47 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java @@ -0,0 +1,38 @@ +/* + * 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.flink.data; + +import java.io.File; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; + +/** + * @deprecated should be removed in 1.8.0; along with FlinkAvroReader. + */ +@Deprecated +public class TestFlinkAvroDeprecatedReaderWriter extends AbstractTestFlinkAvroReaderWriter { + + @Override + protected Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema) { + return Avro.read(Files.localInput(recordsFile)) + .project(schema) + .createReaderFunc(FlinkAvroReader::new); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java new file mode 100644 index 000000000000..102a26a94784 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.io.File; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; + +public class TestFlinkAvroPlannedReaderWriter extends AbstractTestFlinkAvroReaderWriter { + + @Override + protected Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema) { + return Avro.read(Files.localInput(recordsFile)) + .project(schema) + .createResolvingReader(FlinkPlannedAvroReader::create); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index 3b6cf0c58f8f..f76e4c4942bd 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -24,6 +24,8 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; import java.util.Map; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; @@ -32,6 +34,9 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.flink.FlinkSchemaUtil; @@ -41,13 +46,23 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +@ExtendWith(ParameterizedTestExtension.class) public class TestRowProjection { @TempDir private Path temp; + @Parameter(index = 0) + protected Boolean useAvroPlannedReader; + + @Parameters(name = "useAvroPlannedReader={0}") + protected static List parameters() { + return Arrays.asList(new Object[] {Boolean.FALSE}, new Object[] {Boolean.TRUE}); + } + private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) throws IOException { File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); @@ -61,16 +76,23 @@ private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, appender.add(row); } - Iterable records = + Avro.ReadBuilder builder = Avro.read(Files.localInput(file)) .project(readSchema) - .createResolvingReader(FlinkPlannedAvroReader::create) - .build(); + .createReaderFunc(FlinkAvroReader::new); + if (useAvroPlannedReader) { + builder = + Avro.read(Files.localInput(file)) + .project(readSchema) + .createResolvingReader(FlinkPlannedAvroReader::create); + } + + Iterable records = builder.build(); return Iterables.getOnlyElement(records); } - @Test + @TestTemplate public void testFullProjection() throws Exception { Schema schema = new Schema( @@ -85,7 +107,7 @@ public void testFullProjection() throws Exception { assertThat(projected.getString(1)).asString().isEqualTo("test"); } - @Test + @TestTemplate public void testSpecialCharacterProjection() throws Exception { Schema schema = new Schema( @@ -105,7 +127,7 @@ public void testSpecialCharacterProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @Test + @TestTemplate public void testReorderedFullProjection() throws Exception { Schema schema = new Schema( @@ -125,7 +147,7 @@ public void testReorderedFullProjection() throws Exception { assertThat(projected.getLong(1)).isEqualTo(34); } - @Test + @TestTemplate public void testReorderedProjection() throws Exception { Schema schema = new Schema( @@ -147,7 +169,7 @@ public void testReorderedProjection() throws Exception { assertThat(projected.isNullAt(2)).isTrue(); } - @Test + @TestTemplate public void testRenamedAddedField() throws Exception { Schema schema = new Schema( @@ -177,7 +199,7 @@ public void testRenamedAddedField() throws Exception { assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); } - @Test + @TestTemplate public void testEmptyProjection() throws Exception { Schema schema = new Schema( @@ -192,7 +214,7 @@ public void testEmptyProjection() throws Exception { assertThat(projected.getArity()).isEqualTo(0); } - @Test + @TestTemplate public void testBasicProjection() throws Exception { Schema writeSchema = new Schema( @@ -216,7 +238,7 @@ public void testBasicProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @Test + @TestTemplate public void testRename() throws Exception { Schema writeSchema = new Schema( @@ -239,7 +261,7 @@ public void testRename() throws Exception { .isEqualTo("test"); } - @Test + @TestTemplate public void testNestedStructProjection() throws Exception { Schema writeSchema = new Schema( @@ -305,7 +327,7 @@ public void testNestedStructProjection() throws Exception { .isEqualTo(-1.539054f, withPrecision(0.000001f)); } - @Test + @TestTemplate public void testMapProjection() throws IOException { Schema writeSchema = new Schema( @@ -359,7 +381,7 @@ public void testMapProjection() throws IOException { return stringMap; } - @Test + @TestTemplate public void testMapOfStructsProjection() throws IOException { Schema writeSchema = new Schema( @@ -459,7 +481,7 @@ public void testMapOfStructsProjection() throws IOException { .isEqualTo(52.995143f, withPrecision(0.000001f)); } - @Test + @TestTemplate public void testListProjection() throws IOException { Schema writeSchema = new Schema( @@ -488,7 +510,7 @@ public void testListProjection() throws IOException { assertThat(projected.getArray(0)).isEqualTo(values); } - @Test + @TestTemplate @SuppressWarnings("unchecked") public void testListOfStructsProjection() throws IOException { Schema writeSchema = @@ -565,7 +587,7 @@ public void testListOfStructsProjection() throws IOException { assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); } - @Test + @TestTemplate public void testAddedFieldsWithRequiredChildren() throws Exception { Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); From 38d054e4a41ed30be6f7d3480b80150c50b2c3e8 Mon Sep 17 00:00:00 2001 From: Soumya Banerjee <48854046+jeesou@users.noreply.github.com> Date: Wed, 27 Nov 2024 12:32:27 +0530 Subject: [PATCH 197/313] Spark 3.4: Add procedure to compute table stats (#11652) --- .../TestComputeTableStatsProcedure.java | 140 ++++++++++++++++++ .../ComputeTableStatsProcedure.java | 122 +++++++++++++++ .../spark/procedures/SparkProcedures.java | 1 + 3 files changed, 263 insertions(+) create mode 100644 spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java new file mode 100644 index 000000000000..1597c47bd5d3 --- /dev/null +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.BlobMetadata; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.actions.NDVSketchUtil; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.junit.After; +import org.junit.Test; + +public class TestComputeTableStatsProcedure extends SparkExtensionsTestBase { + + public TestComputeTableStatsProcedure( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @After + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @Test + public void testProcedureOnEmptyTable() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + List result = + sql("CALL %s.system.compute_table_stats('%s')", catalogName, tableIdent); + assertThat(result).isEmpty(); + } + + @Test + public void testProcedureWithNamedArgs() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + List output = + sql( + "CALL %s.system.compute_table_stats(table => '%s', columns => array('id'))", + catalogName, tableIdent); + assertThat(output.get(0)).isNotEmpty(); + Object obj = output.get(0)[0]; + assertThat(obj.toString()).endsWith(".stats"); + verifyTableStats(tableName); + } + + @Test + public void testProcedureWithPositionalArgs() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Snapshot snapshot = table.currentSnapshot(); + List output = + sql( + "CALL %s.system.compute_table_stats('%s', %dL)", + catalogName, tableIdent, snapshot.snapshotId()); + assertThat(output.get(0)).isNotEmpty(); + Object obj = output.get(0)[0]; + assertThat(obj.toString()).endsWith(".stats"); + verifyTableStats(tableName); + } + + @Test + public void testProcedureWithInvalidColumns() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_table_stats(table => '%s', columns => array('id1'))", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can't find column id1"); + } + + @Test + public void testProcedureWithInvalidSnapshot() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_table_stats(table => '%s', snapshot_id => %dL)", + catalogName, tableIdent, 1234L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Snapshot not found"); + } + + @Test + public void testProcedureWithInvalidTable() { + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_table_stats(table => '%s', snapshot_id => %dL)", + catalogName, TableIdentifier.of(Namespace.of("default"), "abcd"), 1234L)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Couldn't load table"); + } + + void verifyTableStats(String tableName) throws NoSuchTableException, ParseException { + Table table = Spark3Util.loadIcebergTable(spark, tableName); + StatisticsFile statisticsFile = table.statisticsFiles().get(0); + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties()) + .containsKey(NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java new file mode 100644 index 000000000000..1c2d7125a38a --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.procedures; + +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.ComputeTableStats.Result; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter; +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.apache.spark.unsafe.types.UTF8String; + +/** + * A procedure that computes statistics of a table. + * + * @see SparkActions#computeTableStats(Table) + */ +public class ComputeTableStatsProcedure extends BaseProcedure { + + private static final ProcedureParameter TABLE_PARAM = + ProcedureParameter.required("table", DataTypes.StringType); + private static final ProcedureParameter SNAPSHOT_ID_PARAM = + ProcedureParameter.optional("snapshot_id", DataTypes.LongType); + private static final ProcedureParameter COLUMNS_PARAM = + ProcedureParameter.optional("columns", STRING_ARRAY); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, SNAPSHOT_ID_PARAM, COLUMNS_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("statistics_file", DataTypes.StringType, true, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new Builder() { + @Override + protected ComputeTableStatsProcedure doBuild() { + return new ComputeTableStatsProcedure(tableCatalog()); + } + }; + } + + private ComputeTableStatsProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + Long snapshotId = input.asLong(SNAPSHOT_ID_PARAM, null); + String[] columns = input.asStringArray(COLUMNS_PARAM, null); + + return modifyIcebergTable( + tableIdent, + table -> { + ComputeTableStats action = actions().computeTableStats(table); + + if (snapshotId != null) { + action.snapshot(snapshotId); + } + + if (columns != null) { + action.columns(columns); + } + + Result result = action.execute(); + return toOutputRows(result); + }); + } + + private InternalRow[] toOutputRows(Result result) { + StatisticsFile statisticsFile = result.statisticsFile(); + if (statisticsFile != null) { + InternalRow row = newInternalRow(UTF8String.fromString(statisticsFile.path())); + return new InternalRow[] {row}; + } else { + return new InternalRow[0]; + } + } + + @Override + public String description() { + return "ComputeTableStatsProcedure"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java index 42003b24e94c..d636a21ddc00 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java @@ -61,6 +61,7 @@ private static Map> initProcedureBuilders() { mapBuilder.put("create_changelog_view", CreateChangelogViewProcedure::builder); mapBuilder.put("rewrite_position_delete_files", RewritePositionDeleteFilesProcedure::builder); mapBuilder.put("fast_forward", FastForwardBranchProcedure::builder); + mapBuilder.put("compute_table_stats", ComputeTableStatsProcedure::builder); return mapBuilder.build(); } From 57527d743b0c1863475aca17fe73ca26f28f6f8d Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Wed, 27 Nov 2024 13:43:46 +0100 Subject: [PATCH 198/313] Flink: Backport #11244 to Flink 1.19 (Add table.exec.iceberg.use-v2-sink option) (#11665) --- .../iceberg/flink/FlinkConfigOptions.java | 6 ++ .../iceberg/flink/IcebergTableSink.java | 28 +++++-- .../apache/iceberg/flink/sink/FlinkSink.java | 12 ++- .../iceberg/flink/TestFlinkTableSink.java | 28 ++++++- .../flink/TestFlinkTableSinkExtended.java | 82 +++++++++++++++---- 5 files changed, 123 insertions(+), 33 deletions(-) diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java index 7c7afd24ed8e..6362bc447634 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java @@ -91,6 +91,12 @@ private FlinkConfigOptions() {} .defaultValue(false) .withDescription("Use the FLIP-27 based Iceberg source implementation."); + public static final ConfigOption TABLE_EXEC_ICEBERG_USE_V2_SINK = + ConfigOptions.key("table.exec.iceberg.use-v2-sink") + .booleanType() + .defaultValue(false) + .withDescription("Use the SinkV2 API based Iceberg sink implementation."); + public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = ConfigOptions.key("table.exec.iceberg.split-assigner-type") .enumType(SplitAssignerType.class) diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java index 1b9268569d9a..c2c5a6706e92 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java @@ -35,6 +35,7 @@ import org.apache.flink.types.RowKind; import org.apache.flink.util.Preconditions; import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.flink.sink.IcebergSink; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; public class IcebergTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite { @@ -77,14 +78,25 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { @Override public DataStreamSink consumeDataStream( ProviderContext providerContext, DataStream dataStream) { - return FlinkSink.forRowData(dataStream) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .equalityFieldColumns(equalityColumns) - .overwrite(overwrite) - .setAll(writeProps) - .flinkConf(readableConfig) - .append(); + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK)) { + return IcebergSink.forRowData(dataStream) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .equalityFieldColumns(equalityColumns) + .overwrite(overwrite) + .setAll(writeProps) + .flinkConf(readableConfig) + .append(); + } else { + return FlinkSink.forRowData(dataStream) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .equalityFieldColumns(equalityColumns) + .overwrite(overwrite) + .setAll(writeProps) + .flinkConf(readableConfig) + .append(); + } } }; } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index c53431490984..3f1fef6580ef 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -41,7 +41,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; @@ -381,7 +381,7 @@ public Builder toBranch(String branch) { return this; } - private DataStreamSink chainIcebergOperators() { + private DataStreamSink chainIcebergOperators() { Preconditions.checkArgument( inputCreator != null, "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); @@ -472,12 +472,10 @@ List checkAndGetEqualityFieldIds() { return equalityFieldIds; } - @SuppressWarnings("unchecked") - private DataStreamSink appendDummySink( - SingleOutputStreamOperator committerStream) { - DataStreamSink resultStream = + private DataStreamSink appendDummySink(SingleOutputStreamOperator committerStream) { + DataStreamSink resultStream = committerStream - .addSink(new DiscardingSink()) + .sinkTo(new DiscardingSink<>()) .name(operatorName(String.format("IcebergSink %s", this.table.name()))) .setParallelism(1); if (uidPrefix != null) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index 2978a92945a2..fad30f9c1e67 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -40,7 +40,6 @@ public class TestFlinkTableSink extends CatalogTestBase { - private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; private static final String TABLE_NAME = "test_table"; private TableEnvironment tEnv; private Table icebergTable; @@ -51,7 +50,11 @@ public class TestFlinkTableSink extends CatalogTestBase { @Parameter(index = 3) private boolean isStreamingJob; - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") + @Parameter(index = 4) + private boolean useV2Sink; + + @Parameters( + name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}, useV2Sink={4}") public static List parameters() { List parameters = Lists.newArrayList(); for (FileFormat format : @@ -60,10 +63,24 @@ public static List parameters() { for (Object[] catalogParams : CatalogTestBase.parameters()) { String catalogName = (String) catalogParams[0]; Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); + parameters.add( + new Object[] { + catalogName, baseNamespace, format, isStreaming, false /* don't use v2 sink */ + }); } } } + + for (FileFormat format : + new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { + for (Boolean isStreaming : new Boolean[] {true, false}) { + String catalogName = "testhadoop_basenamespace"; + Namespace baseNamespace = Namespace.of("l0", "l1"); + parameters.add( + new Object[] {catalogName, baseNamespace, format, isStreaming, true /* use v2 sink */}); + } + } + return parameters; } @@ -87,6 +104,11 @@ protected TableEnvironment getTableEnv() { } } } + + tEnv.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK, useV2Sink); + return tEnv; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java index b63547d433a4..3afabf6e0795 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -31,6 +31,8 @@ import java.util.stream.IntStream; import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.apache.flink.streaming.api.transformations.SinkTransformation; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; @@ -52,6 +54,7 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.sink.IcebergSink; import org.apache.iceberg.flink.source.BoundedTableFactory; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -91,11 +94,20 @@ public class TestFlinkTableSinkExtended extends SqlBase { private TableEnvironment tEnv; - @Parameter protected boolean isStreamingJob; + @Parameter(index = 0) + protected boolean isStreamingJob; - @Parameters(name = "isStreamingJob={0}") + @Parameter(index = 1) + protected Boolean useV2Sink; + + @Parameters(name = "isStreamingJob={0}, useV2Sink={1}") protected static List parameters() { - return Arrays.asList(new Boolean[] {true}, new Boolean[] {false}); + return Arrays.asList( + new Object[] {true, false}, + new Object[] {false, false}, + new Object[] {true, true}, + new Object[] {false, true}, + new Object[] {true, null}); } protected synchronized TableEnvironment getTableEnv() { @@ -115,6 +127,13 @@ protected synchronized TableEnvironment getTableEnv() { tEnv = TableEnvironment.create(settingsBuilder.build()); } } + + if (useV2Sink != null) { + tEnv.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_V2_SINK, useV2Sink); + } + return tEnv; } @@ -146,6 +165,32 @@ public void clean() throws Exception { catalog.close(); } + @TestTemplate + public void testUsedFlinkSinkInterface() { + String dataId = BoundedTableFactory.registerDataSet(Collections.emptyList()); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); + String insertSQL = String.format("INSERT INTO %s SELECT * FROM %s", TABLE, SOURCE_TABLE); + ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); + Transformation transformation = + planner.translate(Collections.singletonList(operation)).get(0); + assertThat(transformation).as("Should use SinkV2 API").isInstanceOf(SinkTransformation.class); + SinkTransformation sinkTransformation = (SinkTransformation) transformation; + if (useV2Sink != null && useV2Sink) { + assertThat(sinkTransformation.getSink()) + .as("Should use SinkV2 API based implementation") + .isInstanceOf(IcebergSink.class); + } else { + assertThat(sinkTransformation.getSink()) + .as("Should use custom chain of StreamOperators terminated by DiscardingSink") + .isInstanceOf(DiscardingSink.class); + } + } + @TestTemplate public void testWriteParallelism() { List dataSet = @@ -165,18 +210,25 @@ public void testWriteParallelism() { "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", TABLE, SOURCE_TABLE); ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); - Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); - Transformation committer = dummySink.getInputs().get(0); - Transformation writer = committer.getInputs().get(0); - - assertThat(writer.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); - writer - .getInputs() - .forEach( - input -> - assertThat(input.getParallelism()) - .as("Should have the expected parallelism.") - .isEqualTo(isStreamingJob ? 2 : 4)); + Transformation sink = planner.translate(Collections.singletonList(operation)).get(0); + if (useV2Sink != null && useV2Sink) { + assertThat(sink.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); + Transformation writerInput = sink.getInputs().get(0); + assertThat(writerInput.getParallelism()) + .as("Should have the expected parallelism.") + .isEqualTo(isStreamingJob ? 2 : 4); + } else { + Transformation committer = sink.getInputs().get(0); + Transformation writer = committer.getInputs().get(0); + + assertThat(writer.getParallelism()) + .as("Should have the expected 1 parallelism.") + .isEqualTo(1); + Transformation writerInput = writer.getInputs().get(0); + assertThat(writerInput.getParallelism()) + .as("Should have the expected parallelism.") + .isEqualTo(isStreamingJob ? 2 : 4); + } } @TestTemplate From e9f24f896e4d5fd1e87831f7b95df7b343d0f693 Mon Sep 17 00:00:00 2001 From: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com> Date: Wed, 27 Nov 2024 20:48:10 +0800 Subject: [PATCH 199/313] Doc: Fix some Javadoc URLs. (#11666) --- format/spec.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/format/spec.md b/format/spec.md index 9d8157ae302d..c9cef3290850 100644 --- a/format/spec.md +++ b/format/spec.md @@ -978,7 +978,7 @@ Each version of table metadata is stored in a metadata folder under the table’ Notes: -1. The file system table scheme is implemented in [HadoopTableOperations](../javadoc/{{ icebergVersion }}/index.html?org/apache/iceberg/hadoop/HadoopTableOperations.html). +1. The file system table scheme is implemented in [HadoopTableOperations](../javadoc/{{ icebergVersion }}/org/apache/iceberg/hadoop/HadoopTableOperations.html). #### Metastore Tables @@ -994,7 +994,7 @@ Each version of table metadata is stored in a metadata folder under the table’ Notes: -1. The metastore table scheme is partly implemented in [BaseMetastoreTableOperations](../javadoc/{{ icebergVersion }}/index.html?org/apache/iceberg/BaseMetastoreTableOperations.html). +1. The metastore table scheme is partly implemented in [BaseMetastoreTableOperations](../javadoc/{{ icebergVersion }}/org/apache/iceberg/BaseMetastoreTableOperations.html). ### Delete Formats From 2b3cb5b935996b5a7b4322b6fe051a2bc76bd209 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Wed, 27 Nov 2024 14:52:19 +0100 Subject: [PATCH 200/313] Docs: Add blog post showing Nussknacker with Iceberg integration (#11667) --- site/docs/blogs.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 1cf0b745f6b3..50399f1c84a6 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -58,6 +58,12 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + +### [Using Nussknacker with Apache Iceberg: Periodical report example](https://nussknacker.io/blog/nussknacker-iceberg-example) +**Date**: September 27th, 2024, **Company**: Nussknacker + +**Author**: [Arkadiusz Burdach](https://www.linkedin.com/in/arekburdach/) + ### [Hands-on with Apache Iceberg on Your Laptop: Deep Dive with Apache Spark, Nessie, Minio, Dremio, Polars and Seaborn](https://medium.com/data-engineering-with-dremio/hands-on-with-apache-iceberg-on-your-laptop-deep-dive-with-apache-spark-nessie-minio-dremio-c5d689b01730) **Date**: September 20th, 2024, **Company**: Dremio From 9288d987f6a62b54dbe3cb7bdf61f896e7a8b0b5 Mon Sep 17 00:00:00 2001 From: Hugo Friant Date: Wed, 27 Nov 2024 15:47:51 +0100 Subject: [PATCH 201/313] Kafka Connect: Add config to prefix the control consumer group (#11599) * Add the ability to change the control group prefix * Add doc * Review naming * Review naming * Fix violations --- docs/docs/kafka-connect.md | 1 + .../org/apache/iceberg/connect/IcebergSinkConfig.java | 11 +++++++++++ .../org/apache/iceberg/connect/channel/Worker.java | 2 +- 3 files changed, 13 insertions(+), 1 deletion(-) diff --git a/docs/docs/kafka-connect.md b/docs/docs/kafka-connect.md index a904a17a9968..836f1d88523d 100644 --- a/docs/docs/kafka-connect.md +++ b/docs/docs/kafka-connect.md @@ -77,6 +77,7 @@ for exactly-once semantics. This requires Kafka 2.5 or later. | iceberg.table.\

  • .partition-by | Comma-separated list of partition fields to use when creating the table | | iceberg.table.\
    .route-regex | The regex used to match a record's `routeField` to a table | | iceberg.control.topic | Name of the control topic, default is `control-iceberg` | +| iceberg.control.group-id-prefix | Prefix for the control consumer group, default is `cg-control` | | iceberg.control.commit.interval-ms | Commit interval in msec, default is 300,000 (5 min) | | iceberg.control.commit.timeout-ms | Commit timeout interval in msec, default is 30,000 (30 sec) | | iceberg.control.commit.threads | Number of threads to use for commits, default is (cores * 2) | diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java index bf5b59a0f025..8e59d73923eb 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java @@ -80,6 +80,7 @@ public class IcebergSinkConfig extends AbstractConfig { private static final String TABLES_SCHEMA_CASE_INSENSITIVE_PROP = "iceberg.tables.schema-case-insensitive"; private static final String CONTROL_TOPIC_PROP = "iceberg.control.topic"; + private static final String CONTROL_GROUP_ID_PREFIX_PROP = "iceberg.control.group-id-prefix"; private static final String COMMIT_INTERVAL_MS_PROP = "iceberg.control.commit.interval-ms"; private static final int COMMIT_INTERVAL_MS_DEFAULT = 300_000; private static final String COMMIT_TIMEOUT_MS_PROP = "iceberg.control.commit.timeout-ms"; @@ -180,6 +181,12 @@ private static ConfigDef newConfigDef() { DEFAULT_CONTROL_TOPIC, Importance.MEDIUM, "Name of the control topic"); + configDef.define( + CONTROL_GROUP_ID_PREFIX_PROP, + ConfigDef.Type.STRING, + DEFAULT_CONTROL_GROUP_PREFIX, + Importance.LOW, + "Prefix of the control consumer group"); configDef.define( CONNECT_GROUP_ID_PROP, ConfigDef.Type.STRING, @@ -359,6 +366,10 @@ public String controlTopic() { return getString(CONTROL_TOPIC_PROP); } + public String controlGroupIdPrefix() { + return getString(CONTROL_GROUP_ID_PREFIX_PROP); + } + public String connectGroupId() { String result = getString(CONNECT_GROUP_ID_PROP); if (result != null) { diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java index 7555b216cd45..27c5b9622fd3 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java @@ -51,7 +51,7 @@ class Worker extends Channel { // pass transient consumer group ID to which we never commit offsets super( "worker", - IcebergSinkConfig.DEFAULT_CONTROL_GROUP_PREFIX + UUID.randomUUID(), + config.controlGroupIdPrefix() + UUID.randomUUID(), config, clientFactory, context); From bd7cff17590e285e804234e43086cd0fca18cb4b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Wed, 27 Nov 2024 21:05:46 +0100 Subject: [PATCH 202/313] Flink: Backport Avro planned reader (and corresponding tests) on Flink v1.18 and v1.19 (#11668) --- .../iceberg/flink/data/FlinkAvroReader.java | 12 ++ .../flink/data/FlinkPlannedAvroReader.java | 192 ++++++++++++++++++ .../iceberg/flink/data/FlinkValueReaders.java | 32 +++ .../source/RowDataFileScanTaskReader.java | 4 +- ...=> AbstractTestFlinkAvroReaderWriter.java} | 11 +- .../TestFlinkAvroDeprecatedReaderWriter.java | 38 ++++ .../TestFlinkAvroPlannedReaderWriter.java | 34 ++++ .../iceberg/flink/data/TestRowProjection.java | 58 ++++-- .../iceberg/flink/data/FlinkAvroReader.java | 12 ++ .../flink/data/FlinkPlannedAvroReader.java | 192 ++++++++++++++++++ .../iceberg/flink/data/FlinkValueReaders.java | 32 +++ .../source/RowDataFileScanTaskReader.java | 4 +- ...=> AbstractTestFlinkAvroReaderWriter.java} | 11 +- .../TestFlinkAvroDeprecatedReaderWriter.java | 38 ++++ .../TestFlinkAvroPlannedReaderWriter.java | 34 ++++ .../iceberg/flink/data/TestRowProjection.java | 58 ++++-- 16 files changed, 708 insertions(+), 54 deletions(-) create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java rename flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/{TestFlinkAvroReaderWriter.java => AbstractTestFlinkAvroReaderWriter.java} (96%) create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java rename flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/{TestFlinkAvroReaderWriter.java => AbstractTestFlinkAvroReaderWriter.java} (96%) create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java index 86404959735a..baae91dd1882 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java @@ -37,16 +37,28 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +/** + * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. + */ +@Deprecated public class FlinkAvroReader implements DatumReader, SupportsRowPosition { private final Schema readSchema; private final ValueReader reader; private Schema fileSchema = null; + /** + * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. + */ + @Deprecated public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { this(expectedSchema, readSchema, ImmutableMap.of()); } + /** + * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. + */ + @Deprecated @SuppressWarnings("unchecked") public FlinkAvroReader( org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java new file mode 100644 index 000000000000..b7a81752d4a0 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.avro.AvroWithPartnerVisitor; +import org.apache.iceberg.avro.SupportsRowPosition; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; + +public class FlinkPlannedAvroReader implements DatumReader, SupportsRowPosition { + + private final Types.StructType expectedType; + private final Map idToConstant; + private ValueReader reader; + + public static FlinkPlannedAvroReader create(org.apache.iceberg.Schema schema) { + return create(schema, ImmutableMap.of()); + } + + public static FlinkPlannedAvroReader create( + org.apache.iceberg.Schema schema, Map constants) { + return new FlinkPlannedAvroReader(schema, constants); + } + + private FlinkPlannedAvroReader( + org.apache.iceberg.Schema expectedSchema, Map constants) { + this.expectedType = expectedSchema.asStruct(); + this.idToConstant = constants; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema fileSchema) { + this.reader = + (ValueReader) + AvroWithPartnerVisitor.visit( + expectedType, + fileSchema, + new ReadBuilder(idToConstant), + AvroWithPartnerVisitor.FieldIDAccessors.get()); + } + + @Override + public RowData read(RowData reuse, Decoder decoder) throws IOException { + return reader.read(decoder, reuse); + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + private static class ReadBuilder extends AvroWithPartnerVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public ValueReader record(Type partner, Schema record, List> fieldReaders) { + if (partner == null) { + return ValueReaders.skipStruct(fieldReaders); + } + + Types.StructType expected = partner.asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldReaders, idToConstant); + + // TODO: should this pass expected so that struct.get can reuse containers? + return FlinkValueReaders.struct(readPlan, expected.fields().size()); + } + + @Override + public ValueReader union(Type partner, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader array(Type partner, Schema array, ValueReader elementReader) { + return FlinkValueReaders.array(elementReader); + } + + @Override + public ValueReader arrayMap( + Type partner, Schema map, ValueReader keyReader, ValueReader valueReader) { + return FlinkValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader map(Type partner, Schema map, ValueReader valueReader) { + return FlinkValueReaders.map(FlinkValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Type partner, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + // Flink uses the same representation + return ValueReaders.ints(); + + case "time-micros": + return FlinkValueReaders.timeMicros(); + + case "timestamp-millis": + return FlinkValueReaders.timestampMills(); + + case "timestamp-micros": + return FlinkValueReaders.timestampMicros(); + + case "decimal": + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return FlinkValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + decimal.getPrecision(), + decimal.getScale()); + + case "uuid": + return FlinkValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType.getName()); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + if (partner != null && partner.typeId() == Type.TypeID.LONG) { + return ValueReaders.intsAsLongs(); + } + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + if (partner != null && partner.typeId() == Type.TypeID.DOUBLE) { + return ValueReaders.floatsAsDoubles(); + } + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return FlinkValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive.getFixedSize()); + case BYTES: + return ValueReaders.bytes(); + case ENUM: + return FlinkValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java index 32f6c3a2ccfd..0c6ff2411160 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; public class FlinkValueReaders { @@ -86,6 +87,10 @@ static ValueReader map(ValueReader keyReader, ValueReader valueRe return new MapReader(keyReader, valueReader); } + static ValueReader struct(List>> readPlan, int numFields) { + return new PlannedStructReader(readPlan, numFields); + } + static ValueReader struct( List> readers, Types.StructType struct, Map idToConstant) { return new StructReader(readers, struct, idToConstant); @@ -282,6 +287,33 @@ public MapData read(Decoder decoder, Object reuse) throws IOException { } } + private static class PlannedStructReader extends ValueReaders.PlannedStructReader { + private final int numFields; + + private PlannedStructReader(List>> readPlan, int numFields) { + super(readPlan); + this.numFields = numFields; + } + + @Override + protected RowData reuseOrCreate(Object reuse) { + if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) { + return (RowData) reuse; + } + return new GenericRowData(numFields); + } + + @Override + protected Object get(RowData struct, int pos) { + return null; + } + + @Override + protected void set(RowData struct, int pos, Object value) { + ((GenericRowData) struct).setField(pos, value); + } + } + private static class StructReader extends ValueReaders.StructReader { private final int numFields; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index 88364f4e87b1..9c75a5e0f0fc 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -35,9 +35,9 @@ import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkSourceFilter; import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.FlinkAvroReader; import org.apache.iceberg.flink.data.FlinkOrcReader; import org.apache.iceberg.flink.data.FlinkParquetReaders; +import org.apache.iceberg.flink.data.FlinkPlannedAvroReader; import org.apache.iceberg.flink.data.RowDataProjection; import org.apache.iceberg.flink.data.RowDataUtil; import org.apache.iceberg.io.CloseableIterable; @@ -154,7 +154,7 @@ private CloseableIterable newAvroIterable( .reuseContainers() .project(schema) .split(task.start(), task.length()) - .createReaderFunc(readSchema -> new FlinkAvroReader(schema, readSchema, idToConstant)); + .createReaderFunc(readSchema -> FlinkPlannedAvroReader.create(schema, idToConstant)); if (nameMapping != null) { builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java similarity index 96% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java index a1039d27d888..cbf49ae6faa9 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java @@ -48,7 +48,7 @@ import org.apache.iceberg.util.DateTimeUtil; import org.junit.jupiter.api.Test; -public class TestFlinkAvroReaderWriter extends DataTest { +public abstract class AbstractTestFlinkAvroReaderWriter extends DataTest { private static final int NUM_RECORDS = 100; @@ -70,6 +70,8 @@ protected void writeAndValidate(Schema schema) throws IOException { writeAndValidate(schema, expectedRecords, NUM_RECORDS); } + protected abstract Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema); + private void writeAndValidate(Schema schema, List expectedRecords, int numRecord) throws IOException { RowType flinkSchema = FlinkSchemaUtil.convert(schema); @@ -88,11 +90,7 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n writer.addAll(expectedRecords); } - try (CloseableIterable reader = - Avro.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(FlinkAvroReader::new) - .build()) { + try (CloseableIterable reader = createAvroReadBuilder(recordsFile, schema).build()) { Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < numRecord; i++) { @@ -156,7 +154,6 @@ private Record recordNumType( @Test public void testNumericTypes() throws IOException { - List expected = ImmutableList.of( recordNumType( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java new file mode 100644 index 000000000000..03910f4fda47 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java @@ -0,0 +1,38 @@ +/* + * 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.flink.data; + +import java.io.File; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; + +/** + * @deprecated should be removed in 1.8.0; along with FlinkAvroReader. + */ +@Deprecated +public class TestFlinkAvroDeprecatedReaderWriter extends AbstractTestFlinkAvroReaderWriter { + + @Override + protected Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema) { + return Avro.read(Files.localInput(recordsFile)) + .project(schema) + .createReaderFunc(FlinkAvroReader::new); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java new file mode 100644 index 000000000000..102a26a94784 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.io.File; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; + +public class TestFlinkAvroPlannedReaderWriter extends AbstractTestFlinkAvroReaderWriter { + + @Override + protected Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema) { + return Avro.read(Files.localInput(recordsFile)) + .project(schema) + .createResolvingReader(FlinkPlannedAvroReader::create); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index e76452b7cea0..282a6055cbd3 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -24,6 +24,8 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; import java.util.Map; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; @@ -32,6 +34,9 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.flink.FlinkSchemaUtil; @@ -40,13 +45,23 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +@ExtendWith(ParameterizedTestExtension.class) public class TestRowProjection { @TempDir private Path temp; + @Parameter(index = 0) + protected Boolean useAvroPlannedReader; + + @Parameters(name = "useAvroPlannedReader={0}") + protected static List parameters() { + return Arrays.asList(new Object[] {Boolean.FALSE}, new Object[] {Boolean.TRUE}); + } + private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) throws IOException { File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); @@ -60,16 +75,23 @@ private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, appender.add(row); } - Iterable records = + Avro.ReadBuilder builder = Avro.read(Files.localInput(file)) .project(readSchema) - .createReaderFunc(FlinkAvroReader::new) - .build(); + .createReaderFunc(FlinkAvroReader::new); + if (useAvroPlannedReader) { + builder = + Avro.read(Files.localInput(file)) + .project(readSchema) + .createResolvingReader(FlinkPlannedAvroReader::create); + } + + Iterable records = builder.build(); return Iterables.getOnlyElement(records); } - @Test + @TestTemplate public void testFullProjection() throws Exception { Schema schema = new Schema( @@ -84,7 +106,7 @@ public void testFullProjection() throws Exception { assertThat(projected.getString(1)).asString().isEqualTo("test"); } - @Test + @TestTemplate public void testSpecialCharacterProjection() throws Exception { Schema schema = new Schema( @@ -104,7 +126,7 @@ public void testSpecialCharacterProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @Test + @TestTemplate public void testReorderedFullProjection() throws Exception { Schema schema = new Schema( @@ -124,7 +146,7 @@ public void testReorderedFullProjection() throws Exception { assertThat(projected.getLong(1)).isEqualTo(34); } - @Test + @TestTemplate public void testReorderedProjection() throws Exception { Schema schema = new Schema( @@ -146,7 +168,7 @@ public void testReorderedProjection() throws Exception { assertThat(projected.isNullAt(2)).isTrue(); } - @Test + @TestTemplate public void testRenamedAddedField() throws Exception { Schema schema = new Schema( @@ -176,7 +198,7 @@ public void testRenamedAddedField() throws Exception { assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); } - @Test + @TestTemplate public void testEmptyProjection() throws Exception { Schema schema = new Schema( @@ -191,7 +213,7 @@ public void testEmptyProjection() throws Exception { assertThat(projected.getArity()).isEqualTo(0); } - @Test + @TestTemplate public void testBasicProjection() throws Exception { Schema writeSchema = new Schema( @@ -214,7 +236,7 @@ public void testBasicProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @Test + @TestTemplate public void testRename() throws Exception { Schema writeSchema = new Schema( @@ -237,7 +259,7 @@ public void testRename() throws Exception { .isEqualTo("test"); } - @Test + @TestTemplate public void testNestedStructProjection() throws Exception { Schema writeSchema = new Schema( @@ -303,7 +325,7 @@ public void testNestedStructProjection() throws Exception { .isEqualTo(-1.539054f, withPrecision(0.000001f)); } - @Test + @TestTemplate public void testMapProjection() throws IOException { Schema writeSchema = new Schema( @@ -357,7 +379,7 @@ public void testMapProjection() throws IOException { return stringMap; } - @Test + @TestTemplate public void testMapOfStructsProjection() throws IOException { Schema writeSchema = new Schema( @@ -457,7 +479,7 @@ public void testMapOfStructsProjection() throws IOException { .isEqualTo(52.995143f, withPrecision(0.000001f)); } - @Test + @TestTemplate public void testListProjection() throws IOException { Schema writeSchema = new Schema( @@ -486,7 +508,7 @@ public void testListProjection() throws IOException { assertThat(projected.getArray(0)).isEqualTo(values); } - @Test + @TestTemplate @SuppressWarnings("unchecked") public void testListOfStructsProjection() throws IOException { Schema writeSchema = @@ -563,7 +585,7 @@ public void testListOfStructsProjection() throws IOException { assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); } - @Test + @TestTemplate public void testAddedFieldsWithRequiredChildren() throws Exception { Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java index 86404959735a..baae91dd1882 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java @@ -37,16 +37,28 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +/** + * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. + */ +@Deprecated public class FlinkAvroReader implements DatumReader, SupportsRowPosition { private final Schema readSchema; private final ValueReader reader; private Schema fileSchema = null; + /** + * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. + */ + @Deprecated public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { this(expectedSchema, readSchema, ImmutableMap.of()); } + /** + * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. + */ + @Deprecated @SuppressWarnings("unchecked") public FlinkAvroReader( org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java new file mode 100644 index 000000000000..b7a81752d4a0 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkPlannedAvroReader.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.avro.AvroWithPartnerVisitor; +import org.apache.iceberg.avro.SupportsRowPosition; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; + +public class FlinkPlannedAvroReader implements DatumReader, SupportsRowPosition { + + private final Types.StructType expectedType; + private final Map idToConstant; + private ValueReader reader; + + public static FlinkPlannedAvroReader create(org.apache.iceberg.Schema schema) { + return create(schema, ImmutableMap.of()); + } + + public static FlinkPlannedAvroReader create( + org.apache.iceberg.Schema schema, Map constants) { + return new FlinkPlannedAvroReader(schema, constants); + } + + private FlinkPlannedAvroReader( + org.apache.iceberg.Schema expectedSchema, Map constants) { + this.expectedType = expectedSchema.asStruct(); + this.idToConstant = constants; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema fileSchema) { + this.reader = + (ValueReader) + AvroWithPartnerVisitor.visit( + expectedType, + fileSchema, + new ReadBuilder(idToConstant), + AvroWithPartnerVisitor.FieldIDAccessors.get()); + } + + @Override + public RowData read(RowData reuse, Decoder decoder) throws IOException { + return reader.read(decoder, reuse); + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + private static class ReadBuilder extends AvroWithPartnerVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public ValueReader record(Type partner, Schema record, List> fieldReaders) { + if (partner == null) { + return ValueReaders.skipStruct(fieldReaders); + } + + Types.StructType expected = partner.asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldReaders, idToConstant); + + // TODO: should this pass expected so that struct.get can reuse containers? + return FlinkValueReaders.struct(readPlan, expected.fields().size()); + } + + @Override + public ValueReader union(Type partner, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader array(Type partner, Schema array, ValueReader elementReader) { + return FlinkValueReaders.array(elementReader); + } + + @Override + public ValueReader arrayMap( + Type partner, Schema map, ValueReader keyReader, ValueReader valueReader) { + return FlinkValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader map(Type partner, Schema map, ValueReader valueReader) { + return FlinkValueReaders.map(FlinkValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Type partner, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + // Flink uses the same representation + return ValueReaders.ints(); + + case "time-micros": + return FlinkValueReaders.timeMicros(); + + case "timestamp-millis": + return FlinkValueReaders.timestampMills(); + + case "timestamp-micros": + return FlinkValueReaders.timestampMicros(); + + case "decimal": + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return FlinkValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + decimal.getPrecision(), + decimal.getScale()); + + case "uuid": + return FlinkValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType.getName()); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + if (partner != null && partner.typeId() == Type.TypeID.LONG) { + return ValueReaders.intsAsLongs(); + } + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + if (partner != null && partner.typeId() == Type.TypeID.DOUBLE) { + return ValueReaders.floatsAsDoubles(); + } + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return FlinkValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive.getFixedSize()); + case BYTES: + return ValueReaders.bytes(); + case ENUM: + return FlinkValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java index 32f6c3a2ccfd..0c6ff2411160 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; public class FlinkValueReaders { @@ -86,6 +87,10 @@ static ValueReader map(ValueReader keyReader, ValueReader valueRe return new MapReader(keyReader, valueReader); } + static ValueReader struct(List>> readPlan, int numFields) { + return new PlannedStructReader(readPlan, numFields); + } + static ValueReader struct( List> readers, Types.StructType struct, Map idToConstant) { return new StructReader(readers, struct, idToConstant); @@ -282,6 +287,33 @@ public MapData read(Decoder decoder, Object reuse) throws IOException { } } + private static class PlannedStructReader extends ValueReaders.PlannedStructReader { + private final int numFields; + + private PlannedStructReader(List>> readPlan, int numFields) { + super(readPlan); + this.numFields = numFields; + } + + @Override + protected RowData reuseOrCreate(Object reuse) { + if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) { + return (RowData) reuse; + } + return new GenericRowData(numFields); + } + + @Override + protected Object get(RowData struct, int pos) { + return null; + } + + @Override + protected void set(RowData struct, int pos, Object value) { + ((GenericRowData) struct).setField(pos, value); + } + } + private static class StructReader extends ValueReaders.StructReader { private final int numFields; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index 88364f4e87b1..9c75a5e0f0fc 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -35,9 +35,9 @@ import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkSourceFilter; import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.FlinkAvroReader; import org.apache.iceberg.flink.data.FlinkOrcReader; import org.apache.iceberg.flink.data.FlinkParquetReaders; +import org.apache.iceberg.flink.data.FlinkPlannedAvroReader; import org.apache.iceberg.flink.data.RowDataProjection; import org.apache.iceberg.flink.data.RowDataUtil; import org.apache.iceberg.io.CloseableIterable; @@ -154,7 +154,7 @@ private CloseableIterable newAvroIterable( .reuseContainers() .project(schema) .split(task.start(), task.length()) - .createReaderFunc(readSchema -> new FlinkAvroReader(schema, readSchema, idToConstant)); + .createReaderFunc(readSchema -> FlinkPlannedAvroReader.create(schema, idToConstant)); if (nameMapping != null) { builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java similarity index 96% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java index a1039d27d888..cbf49ae6faa9 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java @@ -48,7 +48,7 @@ import org.apache.iceberg.util.DateTimeUtil; import org.junit.jupiter.api.Test; -public class TestFlinkAvroReaderWriter extends DataTest { +public abstract class AbstractTestFlinkAvroReaderWriter extends DataTest { private static final int NUM_RECORDS = 100; @@ -70,6 +70,8 @@ protected void writeAndValidate(Schema schema) throws IOException { writeAndValidate(schema, expectedRecords, NUM_RECORDS); } + protected abstract Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema); + private void writeAndValidate(Schema schema, List expectedRecords, int numRecord) throws IOException { RowType flinkSchema = FlinkSchemaUtil.convert(schema); @@ -88,11 +90,7 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n writer.addAll(expectedRecords); } - try (CloseableIterable reader = - Avro.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(FlinkAvroReader::new) - .build()) { + try (CloseableIterable reader = createAvroReadBuilder(recordsFile, schema).build()) { Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < numRecord; i++) { @@ -156,7 +154,6 @@ private Record recordNumType( @Test public void testNumericTypes() throws IOException { - List expected = ImmutableList.of( recordNumType( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java new file mode 100644 index 000000000000..03910f4fda47 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java @@ -0,0 +1,38 @@ +/* + * 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.flink.data; + +import java.io.File; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; + +/** + * @deprecated should be removed in 1.8.0; along with FlinkAvroReader. + */ +@Deprecated +public class TestFlinkAvroDeprecatedReaderWriter extends AbstractTestFlinkAvroReaderWriter { + + @Override + protected Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema) { + return Avro.read(Files.localInput(recordsFile)) + .project(schema) + .createReaderFunc(FlinkAvroReader::new); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java new file mode 100644 index 000000000000..102a26a94784 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroPlannedReaderWriter.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.io.File; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; + +public class TestFlinkAvroPlannedReaderWriter extends AbstractTestFlinkAvroReaderWriter { + + @Override + protected Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema) { + return Avro.read(Files.localInput(recordsFile)) + .project(schema) + .createResolvingReader(FlinkPlannedAvroReader::create); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index 7dd4e8759c0e..f76e4c4942bd 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -24,6 +24,8 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; import java.util.Map; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; @@ -32,6 +34,9 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.flink.FlinkSchemaUtil; @@ -41,13 +46,23 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +@ExtendWith(ParameterizedTestExtension.class) public class TestRowProjection { @TempDir private Path temp; + @Parameter(index = 0) + protected Boolean useAvroPlannedReader; + + @Parameters(name = "useAvroPlannedReader={0}") + protected static List parameters() { + return Arrays.asList(new Object[] {Boolean.FALSE}, new Object[] {Boolean.TRUE}); + } + private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) throws IOException { File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); @@ -61,16 +76,23 @@ private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, appender.add(row); } - Iterable records = + Avro.ReadBuilder builder = Avro.read(Files.localInput(file)) .project(readSchema) - .createReaderFunc(FlinkAvroReader::new) - .build(); + .createReaderFunc(FlinkAvroReader::new); + if (useAvroPlannedReader) { + builder = + Avro.read(Files.localInput(file)) + .project(readSchema) + .createResolvingReader(FlinkPlannedAvroReader::create); + } + + Iterable records = builder.build(); return Iterables.getOnlyElement(records); } - @Test + @TestTemplate public void testFullProjection() throws Exception { Schema schema = new Schema( @@ -85,7 +107,7 @@ public void testFullProjection() throws Exception { assertThat(projected.getString(1)).asString().isEqualTo("test"); } - @Test + @TestTemplate public void testSpecialCharacterProjection() throws Exception { Schema schema = new Schema( @@ -105,7 +127,7 @@ public void testSpecialCharacterProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @Test + @TestTemplate public void testReorderedFullProjection() throws Exception { Schema schema = new Schema( @@ -125,7 +147,7 @@ public void testReorderedFullProjection() throws Exception { assertThat(projected.getLong(1)).isEqualTo(34); } - @Test + @TestTemplate public void testReorderedProjection() throws Exception { Schema schema = new Schema( @@ -147,7 +169,7 @@ public void testReorderedProjection() throws Exception { assertThat(projected.isNullAt(2)).isTrue(); } - @Test + @TestTemplate public void testRenamedAddedField() throws Exception { Schema schema = new Schema( @@ -177,7 +199,7 @@ public void testRenamedAddedField() throws Exception { assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); } - @Test + @TestTemplate public void testEmptyProjection() throws Exception { Schema schema = new Schema( @@ -192,7 +214,7 @@ public void testEmptyProjection() throws Exception { assertThat(projected.getArity()).isEqualTo(0); } - @Test + @TestTemplate public void testBasicProjection() throws Exception { Schema writeSchema = new Schema( @@ -216,7 +238,7 @@ public void testBasicProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @Test + @TestTemplate public void testRename() throws Exception { Schema writeSchema = new Schema( @@ -239,7 +261,7 @@ public void testRename() throws Exception { .isEqualTo("test"); } - @Test + @TestTemplate public void testNestedStructProjection() throws Exception { Schema writeSchema = new Schema( @@ -305,7 +327,7 @@ public void testNestedStructProjection() throws Exception { .isEqualTo(-1.539054f, withPrecision(0.000001f)); } - @Test + @TestTemplate public void testMapProjection() throws IOException { Schema writeSchema = new Schema( @@ -359,7 +381,7 @@ public void testMapProjection() throws IOException { return stringMap; } - @Test + @TestTemplate public void testMapOfStructsProjection() throws IOException { Schema writeSchema = new Schema( @@ -459,7 +481,7 @@ public void testMapOfStructsProjection() throws IOException { .isEqualTo(52.995143f, withPrecision(0.000001f)); } - @Test + @TestTemplate public void testListProjection() throws IOException { Schema writeSchema = new Schema( @@ -488,7 +510,7 @@ public void testListProjection() throws IOException { assertThat(projected.getArray(0)).isEqualTo(values); } - @Test + @TestTemplate @SuppressWarnings("unchecked") public void testListOfStructsProjection() throws IOException { Schema writeSchema = @@ -565,7 +587,7 @@ public void testListOfStructsProjection() throws IOException { assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); } - @Test + @TestTemplate public void testAddedFieldsWithRequiredChildren() throws Exception { Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); From 3a8bf57ede8a2499fb2b4a62447a0927e6ca6135 Mon Sep 17 00:00:00 2001 From: hengm3467 <100685635+hengm3467@users.noreply.github.com> Date: Thu, 28 Nov 2024 15:04:04 +0800 Subject: [PATCH 203/313] Docs: Add RisingWave (#11642) * docs: add risingwave * Update docs/docs/risingwave.md Co-authored-by: Fokko Driesprong * Update docs/docs/risingwave.md Co-authored-by: Fokko Driesprong --------- Co-authored-by: Fokko Driesprong --- docs/docs/risingwave.md | 92 +++++++++++++++++++++++++++++++++++++++++ docs/mkdocs.yml | 1 + 2 files changed, 93 insertions(+) create mode 100644 docs/docs/risingwave.md diff --git a/docs/docs/risingwave.md b/docs/docs/risingwave.md new file mode 100644 index 000000000000..6a17f61a2cfa --- /dev/null +++ b/docs/docs/risingwave.md @@ -0,0 +1,92 @@ +--- +title: "RisingWave" +--- + + +# RisingWave + +[RisingWave](https://risingwave.com/) is a Postgres-compatible SQL database designed for real-time event streaming data processing, analysis, and management. It can ingest millions of events per second, continuously join and analyze live data streams with historical tables, serve ad-hoc queries in real-time, and deliver fresh, consistent results. + +## Supported Features + +RisingWave supports batch reads and streaming writes of Apache Iceberg™ tables via its built-in source and sink connectors. For more information, see the [Iceberg source connector documentation](https://docs.risingwave.com/integrations/sources/apache-iceberg) and [Iceberg sink connector documentation](https://docs.risingwave.com/integrations/destinations/apache-iceberg). + +## Table Formats and Warehouse Locations + +Currently, RisingWave only supports the Iceberg V2 table format and S3-compatible object storage as Iceberg warehouse locations. + +## Catalogs + +RisingWave supports the following catalogs: + +- `rest` +- `jdbc` / `sql` +- `glue` +- `storage` +- `hive` + +See [RisingWave’s Iceberg catalog documentation](https://docs.risingwave.com/integrations/destinations/apache-iceberg#catalog) for more details. + +## Getting Started + +### Writing Data to Iceberg Tables + +To write data to an Iceberg table, create a sink in RisingWave. The following example writes data from an existing table or materialized view `rw_data` to an Iceberg table `t1`. + +```sql +CREATE SINK sink_to_iceberg FROM t1 WITH ( + connector = 'iceberg', + type = 'upsert', + primary_key = 'id', + database.name = 'demo_db', + table.name = 't1', + catalog.name = 'demo', + catalog.type = 'storage', + warehouse.path = 's3a://hummock001/demo', + s3.endpoint = '', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin' +); +``` + +**Note**: From RisingWave 2.1, you can use the `create_table_if_not_exists` parameter to create a table if it doesn't exist. + +### Reading from Iceberg Tables + +To read data from an Iceberg table, create a source in RisingWave. The following example reads data from an Iceberg table `t1`. + +```sql +CREATE SOURCE iceberg_t1_source WITH ( + connector = 'iceberg', + s3.endpoint = '', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + s3.path.style.access = 'true', + catalog.type = 'storage', + warehouse.path = 's3a://hummock001/demo', + database.name = 'demo_db', + table.name = 't1', +); +``` +After this source is created, you can query the data using the following SQL statement: + +```sql +SELECT * FROM iceberg_t1_source; +``` \ No newline at end of file diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index 87e3fdd8adf9..8d01fba0e3be 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -53,6 +53,7 @@ nav: - hive.md - Trino: https://trino.io/docs/current/connector/iceberg.html - Daft: daft.md + - RisingWave: risingwave.md - ClickHouse: https://clickhouse.com/docs/en/engines/table-engines/integrations/iceberg - Presto: https://prestodb.io/docs/current/connector/iceberg.html - Dremio: https://docs.dremio.com/data-formats/apache-iceberg/ From 163e2068f96f139632488f36928bf443c9be326f Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Thu, 28 Nov 2024 15:14:57 +0530 Subject: [PATCH 204/313] REST: Docker file for REST Catalog Fixture (#11283) * REST: Docker file for Rest catalog adapter image * Address comments * Add cloud bundles * update notice --- build.gradle | 18 +- docker/iceberg-rest-adapter-image/Dockerfile | 42 +++ docker/iceberg-rest-adapter-image/README.md | 86 +++++ open-api/LICENSE | 341 ++++++++++++++++++ open-api/NOTICE | 33 +- .../testFixtures/resources/log4j.properties | 22 ++ 6 files changed, 537 insertions(+), 5 deletions(-) create mode 100644 docker/iceberg-rest-adapter-image/Dockerfile create mode 100644 docker/iceberg-rest-adapter-image/README.md create mode 100644 open-api/src/testFixtures/resources/log4j.properties diff --git a/build.gradle b/build.gradle index 81daf14a357f..eb63ce138ab2 100644 --- a/build.gradle +++ b/build.gradle @@ -948,19 +948,33 @@ project(':iceberg-open-api') { exclude group: 'org.apache.curator' exclude group: 'org.apache.zookeeper' exclude group: 'org.apache.kerby' - exclude group: 'org.apache.hadoop', module: 'hadoop-auth' - exclude group: 'org.apache.commons', module: 'commons-configuration2' exclude group: 'org.apache.hadoop.thirdparty', module: 'hadoop-shaded-protobuf_3_7' exclude group: 'org.eclipse.jetty' + exclude group: 'com.google.re2j', module: 're2j' + exclude group: 'com.google.code.gson', module: 'gson' + exclude group: 'com.jcraft', module: 'jsch' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'io.dropwizard.metrics', module: 'metrics-core' + exclude group: 'dnsjava', module: 'dnsjava' + exclude group: 'org.xerial.snappy', module: 'snappy-java' + exclude group: 'commons-cli', module: 'commons-cli' + exclude group: 'com.github.pjfanning', module: 'jersey-json' } testFixturesImplementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') testFixturesImplementation libs.junit.jupiter + testFixturesImplementation libs.slf4j.api + testFixturesImplementation libs.slf4j.simple + testFixturesImplementation libs.jetty.servlet testFixturesImplementation libs.jetty.server testFixturesImplementation libs.sqlite.jdbc testFixturesCompileOnly libs.apiguardian + + testFixturesRuntimeOnly project(':iceberg-aws-bundle') + testFixturesRuntimeOnly project(':iceberg-azure-bundle') + testFixturesRuntimeOnly project(':iceberg-gcp-bundle') } test { diff --git a/docker/iceberg-rest-adapter-image/Dockerfile b/docker/iceberg-rest-adapter-image/Dockerfile new file mode 100644 index 000000000000..30189b97c53e --- /dev/null +++ b/docker/iceberg-rest-adapter-image/Dockerfile @@ -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. +# + +FROM azul/zulu-openjdk:17-jre-headless + +# Set up the user and group +RUN set -xeu && \ + groupadd iceberg --gid 1000 && \ + useradd iceberg --uid 1000 --gid 1000 --create-home + +# Working directory for the application +WORKDIR /usr/lib/iceberg-rest + +# Copy the JAR file directly to the target location +COPY --chown=iceberg:iceberg open-api/build/libs/iceberg-open-api-test-fixtures-runtime-*.jar /usr/lib/iceberg-rest/iceberg-rest-adapter.jar + +ENV CATALOG_CATALOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog +ENV CATALOG_URI=jdbc:sqlite::memory: +ENV CATALOG_JDBC_USER=user +ENV CATALOG_JDBC_PASSWORD=password +ENV REST_PORT=8181 + +EXPOSE $REST_PORT +USER iceberg:iceberg +ENV LANG=en_US.UTF-8 +CMD ["java", "-jar", "iceberg-rest-adapter.jar"] diff --git a/docker/iceberg-rest-adapter-image/README.md b/docker/iceberg-rest-adapter-image/README.md new file mode 100644 index 000000000000..661b5397e241 --- /dev/null +++ b/docker/iceberg-rest-adapter-image/README.md @@ -0,0 +1,86 @@ + + +# Iceberg REST Catalog Adapter Test Fixture + +For converting different catalog implementations into a rest one. +Adapter for wrapping the existing catalog backends over REST. + + +## Build the Docker Image + +When making changes to the local files and test them out, you can build the image locally: + +```bash +# Build the project from iceberg root directory +./gradlew :iceberg-open-api:shadowJar + +# Rebuild the docker image +docker image rm -f apache/iceberg-rest-adapter && docker build -t apache/iceberg-rest-adapter -f docker/iceberg-rest-adapter-image/Dockerfile . +``` + +## Browse + +To browse the catalog, you can use `pyiceberg`: + +``` +➜ ~ pyiceberg --uri http://localhost:8181 list +default +nyc +ride_sharing_dataset +➜ ~ pyiceberg --uri http://localhost:8181 list nyc +nyc.taxis +nyc.taxis3 +nyc.taxis4 +nyc.taxis_copy_maybe +nyc.taxis_skeleton +nyc.taxis_skeleton2 +➜ ~ pyiceberg --uri http://localhost:8181 describe --entity=table tpcds_iceberg.customer +Table format version 2 +Metadata location s3://iceberg-test-data/tpc/tpc-ds/3.2.0/1000/iceberg/customer/metadata/00001-1bccfcc4-69f6-4505-8df5-4de78356e327.metadata.json +Table UUID dce215f7-6301-4a73-acc4-6e12db016abb +Last Updated 1653550004061 +Partition spec [] +Sort order [] +Schema Schema + ├── 1: c_customer_sk: optional int + ├── 2: c_customer_id: optional string + ├── 3: c_current_cdemo_sk: optional int + ├── 4: c_current_hdemo_sk: optional int + ├── 5: c_current_addr_sk: optional int + ├── 6: c_first_shipto_date_sk: optional int + ├── 7: c_first_sales_date_sk: optional int + ├── 8: c_salutation: optional string + ├── 9: c_first_name: optional string + ├── 10: c_last_name: optional string + ├── 11: c_preferred_cust_flag: optional string + ├── 12: c_birth_day: optional int + ├── 13: c_birth_month: optional int + ├── 14: c_birth_year: optional int + ├── 15: c_birth_country: optional string + ├── 16: c_login: optional string + ├── 17: c_email_address: optional string + └── 18: c_last_review_date: optional string +Snapshots Snapshots + └── Snapshot 0: s3://iceberg-test-data/tpc/tpc-ds/3.2.0/1000/iceberg/customer/metadata/snap-643656366840285027-1-5ce13497-7330-4d02-8206-7e313e43209c.avro +Properties write.object-storage.enabled true + write.object-storage.path s3://iceberg-test-data/tpc/tpc-ds/3.2.0/1000/iceberg/customer/data +``` + + diff --git a/open-api/LICENSE b/open-api/LICENSE index ee4389e3ef8e..c5526b28b99c 100644 --- a/open-api/LICENSE +++ b/open-api/LICENSE @@ -415,11 +415,22 @@ License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt -------------------------------------------------------------------------------- +Group: org.apache.commons Name: commons-configuration2 Version: 2.10.1 +Project URL: https://commons.apache.org/proper/commons-configuration/ +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + Group: org.apache.hadoop Name: hadoop-common Version: 3.3.6 License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt -------------------------------------------------------------------------------- +Group: org.apache.hadoop Name: hadoop-auth Version: 3.3.6 +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + Group: org.apache.httpcomponents.client5 Name: httpclient5 Version: 5.4 License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt @@ -552,4 +563,334 @@ Group: org.xerial Name: sqlite-jdbc Version: 3.47.0.0 Project URL (from POM): https://github.com/xerial/sqlite-jdbc License (from POM): The Apache Software License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: annotations Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: apache-client Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: arns Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: auth Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-core Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: checksums Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: checksums-spi Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: crt-core Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: dynamodb Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: endpoints-spi Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: glue Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth-aws Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-auth-spi Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-client-spi Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: iam Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: identity-spi Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: json-utils Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: kms Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: lakeformation Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: metrics-spi Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: netty-nio-client Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: profiles Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: protocol-core Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: regions Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: retries Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: retries-spi Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: s3 Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sdk-core Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sso Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sts Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.29.6 +Project URL (from POM): https://aws.amazon.com/sdkforjava +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: utils Version: 2.29.6 +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.eventstream Name: eventstream Version: 1.0.1 +Project URL (from POM): https://github.com/awslabs/aws-eventstream-java +License (from POM): Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: api-common Version: 2.40.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/api-common-java/blob/main/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: gax Version: 2.57.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/gax-java/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: gax-grpc Version: 2.57.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/gax-java/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api Name: gax-httpjson Version: 2.57.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD-3-Clause - https://github.com/googleapis/gax-java/blob/master/LICENSE + +-------------------------------------------------------------------------------- + +Group: com.google.api-client Name: google-api-client Version: 2.7.0 +Project URL (from manifest): https://developers.google.com/api-client-library/java/ +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: gapic-google-cloud-storage-v2 Version: 2.44.1-beta +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: grpc-google-cloud-storage-v2 Version: 2.44.1-beta +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: proto-google-cloud-storage-v2 Version: 2.44.1-beta +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: proto-google-common-protos Version: 2.48.0 +Project URL (from POM): https://github.com/googleapis/sdk-platform-java +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.api.grpc Name: proto-google-iam-v1 Version: 1.43.0 +Project URL (from POM): https://github.com/googleapis/sdk-platform-java +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.apis Name: google-api-services-storage Version: v1-rev20241008-2.0.0 +License (from POM): The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.auth Name: google-auth-library-credentials Version: 1.29.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD New license - http://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.auth Name: google-auth-library-oauth2-http Version: 1.29.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt +License (from POM): BSD New license - http://opensource.org/licenses/BSD-3-Clause + +-------------------------------------------------------------------------------- + +Group: com.google.auto.value Name: auto-value-annotations Version: 1.11.0 +Project URL (from POM): https://github.com/google/auto/tree/main/value +License (from POM): Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-core Version: 2.47.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-core-grpc Version: 2.47.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-core-http Version: 2.47.0 +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +Group: com.google.cloud Name: google-cloud-storage Version: 2.47.1 +Project URL (from POM): https://github.com/googleapis/java-storage +License (from POM): Apache-2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- +Group: com.microsoft.azure Name: msal4j Version: 1.17.2 +Project URL (from manifest): https://github.com/AzureAD/microsoft-authentication-library-for-java +Manifest License: "MIT License" (Not packaged) +Project URL (from POM): https://github.com/AzureAD/microsoft-authentication-library-for-java +License (from POM): MIT License +-------------------------------------------------------------------------------- + +Group: com.microsoft.azure Name: msal4j-persistence-extension Version: 1.3.0 +Project URL (from POM): https://github.com/AzureAD/microsoft-authentication-library-for-java +License (from POM): MIT License -------------------------------------------------------------------------------- \ No newline at end of file diff --git a/open-api/NOTICE b/open-api/NOTICE index 15e0d63f9135..30b72252ee47 100644 --- a/open-api/NOTICE +++ b/open-api/NOTICE @@ -63,33 +63,60 @@ See FastDoubleParser-NOTICE for details of other source code included in FastDou and the licenses and copyrights that apply to that code. -------------------------------------------------------------------------------- -This product includes software developed at -The Apache Software Foundation (https://www.apache.org/). - Apache Commons Codec Copyright 2002-2024 The Apache Software Foundation +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + Apache Commons IO Copyright 2002-2024 The Apache Software Foundation +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + Apache Avro Copyright 2009-2024 The Apache Software Foundation +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + Apache Commons Compress Copyright 2002-2024 The Apache Software Foundation +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + Apache Commons Lang Copyright 2001-2023 The Apache Software Foundation +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + +Apache Commons Configuration +Copyright 2001-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + Apache HttpClient Copyright 1999-2021 The Apache Software Foundation +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + Apache HttpComponents Core HTTP/1.1 Copyright 2005-2021 The Apache Software Foundation +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + Apache HttpComponents Core HTTP/2 Copyright 2005-2021 The Apache Software Foundation +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + -------------------------------------------------------------------------------- Apache Hadoop diff --git a/open-api/src/testFixtures/resources/log4j.properties b/open-api/src/testFixtures/resources/log4j.properties new file mode 100644 index 000000000000..30c07034cbe4 --- /dev/null +++ b/open-api/src/testFixtures/resources/log4j.properties @@ -0,0 +1,22 @@ +# 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. + +log4j.rootLogger=INFO, stdout +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd'T'HH:mm:ss.SSS} %-5p [%c] - %m%n \ No newline at end of file From a95943e5561c78c18062852e7f8027a191562e08 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 28 Nov 2024 15:59:42 +0100 Subject: [PATCH 205/313] Core: Propagate custom metrics reporter when table is created/replaced through Transaction (#11671) --- .../apache/iceberg/BaseMetastoreCatalog.java | 9 +- .../java/org/apache/iceberg/Transactions.java | 6 ++ .../iceberg/inmemory/InMemoryCatalog.java | 7 ++ .../apache/iceberg/catalog/CatalogTests.java | 90 +++++++++++++++++++ .../iceberg/inmemory/TestInMemoryCatalog.java | 12 ++- .../apache/iceberg/jdbc/TestJdbcCatalog.java | 50 +---------- .../jdbc/TestJdbcCatalogWithV1Schema.java | 32 ++++--- .../apache/iceberg/rest/TestRESTCatalog.java | 83 ++++------------- .../apache/iceberg/hive/TestHiveCatalog.java | 28 ++++-- .../iceberg/nessie/TestNessieCatalog.java | 18 +++- .../RESTCompatibilityKitCatalogTests.java | 6 ++ 11 files changed, 198 insertions(+), 143 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index e794b3121dc3..e960fe2b63e0 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -217,7 +217,8 @@ public Transaction createTransaction() { tableProperties.putAll(tableOverrideProperties()); TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, sortOrder, baseLocation, tableProperties); - return Transactions.createTableTransaction(identifier.toString(), ops, metadata); + return Transactions.createTableTransaction( + identifier.toString(), ops, metadata, metricsReporter()); } @Override @@ -249,9 +250,11 @@ private Transaction newReplaceTableTransaction(boolean orCreate) { } if (orCreate) { - return Transactions.createOrReplaceTableTransaction(identifier.toString(), ops, metadata); + return Transactions.createOrReplaceTableTransaction( + identifier.toString(), ops, metadata, metricsReporter()); } else { - return Transactions.replaceTableTransaction(identifier.toString(), ops, metadata); + return Transactions.replaceTableTransaction( + identifier.toString(), ops, metadata, metricsReporter()); } } diff --git a/core/src/main/java/org/apache/iceberg/Transactions.java b/core/src/main/java/org/apache/iceberg/Transactions.java index 7afed0573a39..a8ea40a6b90b 100644 --- a/core/src/main/java/org/apache/iceberg/Transactions.java +++ b/core/src/main/java/org/apache/iceberg/Transactions.java @@ -30,6 +30,12 @@ public static Transaction createOrReplaceTableTransaction( return new BaseTransaction(tableName, ops, TransactionType.CREATE_OR_REPLACE_TABLE, start); } + public static Transaction createOrReplaceTableTransaction( + String tableName, TableOperations ops, TableMetadata start, MetricsReporter reporter) { + return new BaseTransaction( + tableName, ops, TransactionType.CREATE_OR_REPLACE_TABLE, start, reporter); + } + public static Transaction replaceTableTransaction( String tableName, TableOperations ops, TableMetadata start) { return new BaseTransaction(tableName, ops, TransactionType.REPLACE_TABLE, start); diff --git a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java index a880f94f4385..ff71bde71ff5 100644 --- a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java +++ b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java @@ -70,6 +70,7 @@ public class InMemoryCatalog extends BaseMetastoreViewCatalog private String catalogName; private String warehouseLocation; private CloseableGroup closeableGroup; + private Map catalogProperties; public InMemoryCatalog() { this.namespaces = Maps.newConcurrentMap(); @@ -85,6 +86,7 @@ public String name() { @Override public void initialize(String name, Map properties) { this.catalogName = name != null ? name : InMemoryCatalog.class.getSimpleName(); + this.catalogProperties = ImmutableMap.copyOf(properties); String warehouse = properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, ""); this.warehouseLocation = warehouse.replaceAll("/*$", ""); @@ -368,6 +370,11 @@ public void renameView(TableIdentifier from, TableIdentifier to) { } } + @Override + protected Map properties() { + return catalogProperties == null ? ImmutableMap.of() : catalogProperties; + } + private class InMemoryTableOperations extends BaseMetastoreTableOperations { private final FileIO fileIO; private final TableIdentifier tableIdentifier; diff --git a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java index a011578865b4..4df91a49033d 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -30,11 +30,14 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.FilesTable; import org.apache.iceberg.HasTableOperations; @@ -56,6 +59,10 @@ import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.MetricsReporter; +import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; @@ -144,6 +151,8 @@ public abstract class CatalogTests { protected abstract C catalog(); + protected abstract C initCatalog(String catalogName, Map additionalProperties); + protected boolean supportsNamespaceProperties() { return true; } @@ -2695,6 +2704,87 @@ public void testRegisterExistingTable() { assertThat(catalog.dropTable(identifier)).isTrue(); } + @Test + public void testCatalogWithCustomMetricsReporter() throws IOException { + C catalogWithCustomReporter = + initCatalog( + "catalog_with_custom_reporter", + ImmutableMap.of( + CatalogProperties.METRICS_REPORTER_IMPL, CustomMetricsReporter.class.getName())); + + if (requiresNamespaceCreate()) { + catalogWithCustomReporter.createNamespace(TABLE.namespace()); + } + + catalogWithCustomReporter.buildTable(TABLE, SCHEMA).create(); + + Table table = catalogWithCustomReporter.loadTable(TABLE); + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(FileFormat.PARQUET.addExtension(UUID.randomUUID().toString())) + .withFileSizeInBytes(10) + .withRecordCount(2) + .build(); + + // append file through FastAppend and check and reset counter + table.newFastAppend().appendFile(dataFile).commit(); + assertThat(CustomMetricsReporter.COMMIT_COUNTER.get()).isEqualTo(1); + CustomMetricsReporter.COMMIT_COUNTER.set(0); + + TableIdentifier identifier = TableIdentifier.of(NS, "custom_metrics_reporter_table"); + // append file through createTransaction() and check and reset counter + catalogWithCustomReporter + .buildTable(identifier, SCHEMA) + .createTransaction() + .newFastAppend() + .appendFile(dataFile) + .commit(); + assertThat(CustomMetricsReporter.COMMIT_COUNTER.get()).isEqualTo(1); + CustomMetricsReporter.COMMIT_COUNTER.set(0); + + // append file through createOrReplaceTransaction() and check and reset counter + catalogWithCustomReporter + .buildTable(identifier, SCHEMA) + .createOrReplaceTransaction() + .newFastAppend() + .appendFile(dataFile) + .commit(); + assertThat(CustomMetricsReporter.COMMIT_COUNTER.get()).isEqualTo(1); + CustomMetricsReporter.COMMIT_COUNTER.set(0); + + // append file through replaceTransaction() and check and reset counter + catalogWithCustomReporter + .buildTable(TABLE, SCHEMA) + .replaceTransaction() + .newFastAppend() + .appendFile(dataFile) + .commit(); + assertThat(CustomMetricsReporter.COMMIT_COUNTER.get()).isEqualTo(1); + CustomMetricsReporter.COMMIT_COUNTER.set(0); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + assertThat(tasks.iterator()).hasNext(); + } + + assertThat(CustomMetricsReporter.SCAN_COUNTER.get()).isEqualTo(1); + // reset counter in case subclasses run this test multiple times + CustomMetricsReporter.SCAN_COUNTER.set(0); + } + + public static class CustomMetricsReporter implements MetricsReporter { + static final AtomicInteger SCAN_COUNTER = new AtomicInteger(0); + static final AtomicInteger COMMIT_COUNTER = new AtomicInteger(0); + + @Override + public void report(MetricsReport report) { + if (report instanceof ScanReport) { + SCAN_COUNTER.incrementAndGet(); + } else if (report instanceof CommitReport) { + COMMIT_COUNTER.incrementAndGet(); + } + } + } + private static void assertEmpty(String context, Catalog catalog, Namespace ns) { try { assertThat(catalog.listTables(ns)).as(context).isEmpty(); diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryCatalog.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryCatalog.java index 63cd24b4e2c6..2c8650d6358b 100644 --- a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryCatalog.java +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryCatalog.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.inmemory; +import java.util.Map; import org.apache.iceberg.catalog.CatalogTests; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; @@ -27,8 +28,7 @@ public class TestInMemoryCatalog extends CatalogTests { @BeforeEach public void before() { - this.catalog = new InMemoryCatalog(); - this.catalog.initialize("in-memory-catalog", ImmutableMap.of()); + this.catalog = initCatalog("in-memory-catalog", ImmutableMap.of()); } @Override @@ -36,6 +36,14 @@ protected InMemoryCatalog catalog() { return catalog; } + @Override + protected InMemoryCatalog initCatalog( + String catalogName, Map additionalProperties) { + InMemoryCatalog cat = new InMemoryCatalog(); + cat.initialize(catalogName, additionalProperties); + return cat; + } + @Override protected boolean requiresNamespaceCreate() { return true; diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java index d21605cace21..2d4eb2f15738 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -39,7 +39,6 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; @@ -50,8 +49,6 @@ import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; @@ -68,9 +65,6 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.hadoop.Util; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.metrics.MetricsReport; -import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -139,7 +133,8 @@ public void setupTable() throws Exception { catalog = initCatalog("test_jdbc_catalog", Maps.newHashMap()); } - private JdbcCatalog initCatalog(String catalogName, Map props) { + @Override + protected JdbcCatalog initCatalog(String catalogName, Map additionalProperties) { Map properties = Maps.newHashMap(); properties.put( CatalogProperties.URI, @@ -150,7 +145,7 @@ private JdbcCatalog initCatalog(String catalogName, Map props) { warehouseLocation = this.tableDir.toAbsolutePath().toString(); properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); properties.put("type", "jdbc"); - properties.putAll(props); + properties.putAll(additionalProperties); return (JdbcCatalog) CatalogUtil.buildIcebergCatalog(catalogName, properties, conf); } @@ -1059,36 +1054,6 @@ public void testConversions() { assertThat(JdbcUtil.stringToNamespace(nsString)).isEqualTo(ns); } - @Test - public void testCatalogWithCustomMetricsReporter() throws IOException { - JdbcCatalog catalogWithCustomReporter = - initCatalog( - "test_jdbc_catalog_with_custom_reporter", - ImmutableMap.of( - CatalogProperties.METRICS_REPORTER_IMPL, CustomMetricsReporter.class.getName())); - try { - catalogWithCustomReporter.buildTable(TABLE, SCHEMA).create(); - Table table = catalogWithCustomReporter.loadTable(TABLE); - table - .newFastAppend() - .appendFile( - DataFiles.builder(PartitionSpec.unpartitioned()) - .withPath(FileFormat.PARQUET.addExtension(UUID.randomUUID().toString())) - .withFileSizeInBytes(10) - .withRecordCount(2) - .build()) - .commit(); - try (CloseableIterable tasks = table.newScan().planFiles()) { - assertThat(tasks.iterator()).hasNext(); - } - } finally { - catalogWithCustomReporter.dropTable(TABLE); - } - // counter of custom metrics reporter should have been increased - // 1x for commit metrics / 1x for scan metrics - assertThat(CustomMetricsReporter.COUNTER.get()).isEqualTo(2); - } - @Test public void testCommitExceptionWithoutMessage() { TableIdentifier tableIdent = TableIdentifier.of("db", "tbl"); @@ -1129,15 +1094,6 @@ public void testCommitExceptionWithMessage() { } } - public static class CustomMetricsReporter implements MetricsReporter { - static final AtomicInteger COUNTER = new AtomicInteger(0); - - @Override - public void report(MetricsReport report) { - COUNTER.incrementAndGet(); - } - } - private String createMetadataLocationViaJdbcCatalog(TableIdentifier identifier) throws SQLException { // temporary connection just to actually create a concrete metadata location diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalogWithV1Schema.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalogWithV1Schema.java index b47c216ffced..7586d880c188 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalogWithV1Schema.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalogWithV1Schema.java @@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.catalog.CatalogTests; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.io.TempDir; @@ -38,6 +39,24 @@ protected JdbcCatalog catalog() { return catalog; } + @Override + protected JdbcCatalog initCatalog(String catalogName, Map additionalProperties) { + Map properties = Maps.newHashMap(); + properties.put( + CatalogProperties.URI, + "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", "")); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, tableDir.toAbsolutePath().toString()); + properties.put(JdbcUtil.SCHEMA_VERSION_PROPERTY, JdbcUtil.SchemaVersion.V1.name()); + properties.putAll(additionalProperties); + + JdbcCatalog cat = new JdbcCatalog(); + cat.setConf(new Configuration()); + cat.initialize(catalogName, properties); + return cat; + } + @Override protected boolean supportsNamespaceProperties() { return true; @@ -50,17 +69,6 @@ protected boolean supportsNestedNamespaces() { @BeforeEach public void setupCatalog() { - Map properties = Maps.newHashMap(); - properties.put( - CatalogProperties.URI, - "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", "")); - properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); - properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, tableDir.toAbsolutePath().toString()); - properties.put(JdbcUtil.SCHEMA_VERSION_PROPERTY, JdbcUtil.SchemaVersion.V1.name()); - - catalog = new JdbcCatalog(); - catalog.setConf(new Configuration()); - catalog.initialize("testCatalog", properties); + this.catalog = initCatalog("testCatalog", ImmutableMap.of()); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 06008761eac1..232cfd31d1a6 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -36,14 +36,12 @@ import java.util.Optional; import java.util.UUID; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTransaction; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataUpdate; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -64,9 +62,6 @@ import org.apache.iceberg.exceptions.ServiceFailureException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.inmemory.InMemoryCatalog; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.metrics.MetricsReport; -import org.apache.iceberg.metrics.MetricsReporter; 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.Maps; @@ -114,7 +109,6 @@ public class TestRESTCatalog extends CatalogTests { @BeforeEach public void createCatalog() throws Exception { File warehouse = temp.toFile(); - Configuration conf = new Configuration(); this.backendCatalog = new InMemoryCatalog(); this.backendCatalog.initialize( @@ -164,6 +158,12 @@ public T execute( httpServer.setHandler(servletContext); httpServer.start(); + this.restCatalog = initCatalog("prod", ImmutableMap.of()); + } + + @Override + protected RESTCatalog initCatalog(String catalogName, Map additionalProperties) { + Configuration conf = new Configuration(); SessionCatalog.SessionContext context = new SessionCatalog.SessionContext( UUID.randomUUID().toString(), @@ -171,20 +171,26 @@ public T execute( ImmutableMap.of("credential", "user:12345"), ImmutableMap.of()); - this.restCatalog = + RESTCatalog catalog = new RESTCatalog( context, (config) -> HTTPClient.builder(config).uri(config.get(CatalogProperties.URI)).build()); - restCatalog.setConf(conf); - restCatalog.initialize( - "prod", + catalog.setConf(conf); + Map properties = ImmutableMap.of( CatalogProperties.URI, httpServer.getURI().toString(), CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.inmemory.InMemoryFileIO", "credential", - "catalog:12345")); + "catalog:12345"); + catalog.initialize( + catalogName, + ImmutableMap.builder() + .putAll(properties) + .putAll(additionalProperties) + .build()); + return catalog; } @SuppressWarnings("unchecked") @@ -1623,61 +1629,6 @@ public void testCatalogRefreshedTokenIsUsed(String oauth2ServerUri) { }); } - @Test - public void testCatalogWithCustomMetricsReporter() throws IOException { - this.restCatalog = - new RESTCatalog( - new SessionCatalog.SessionContext( - UUID.randomUUID().toString(), - "user", - ImmutableMap.of("credential", "user:12345"), - ImmutableMap.of()), - (config) -> HTTPClient.builder(config).uri(config.get(CatalogProperties.URI)).build()); - restCatalog.setConf(new Configuration()); - restCatalog.initialize( - "prod", - ImmutableMap.of( - CatalogProperties.URI, - httpServer.getURI().toString(), - "credential", - "catalog:12345", - CatalogProperties.METRICS_REPORTER_IMPL, - CustomMetricsReporter.class.getName())); - - if (requiresNamespaceCreate()) { - restCatalog.createNamespace(TABLE.namespace()); - } - - restCatalog.buildTable(TABLE, SCHEMA).create(); - Table table = restCatalog.loadTable(TABLE); - table - .newFastAppend() - .appendFile( - DataFiles.builder(PartitionSpec.unpartitioned()) - .withPath("/path/to/data-a.parquet") - .withFileSizeInBytes(10) - .withRecordCount(2) - .build()) - .commit(); - - try (CloseableIterable tasks = table.newScan().planFiles()) { - assertThat(tasks.iterator()).hasNext(); - } - - // counter of custom metrics reporter should have been increased - // 1x for commit metrics / 1x for scan metrics - assertThat(CustomMetricsReporter.COUNTER.get()).isEqualTo(2); - } - - public static class CustomMetricsReporter implements MetricsReporter { - static final AtomicInteger COUNTER = new AtomicInteger(0); - - @Override - public void report(MetricsReport report) { - COUNTER.incrementAndGet(); - } - } - @Test public void testCatalogExpiredBearerTokenRefreshWithoutCredential() { // expires at epoch second = 1 diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java index 7d0eb641a385..709bb1caaa62 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java @@ -110,20 +110,30 @@ public class TestHiveCatalog extends CatalogTests { @BeforeEach public void before() throws TException { - catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), - CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE, - ImmutableMap.of( - CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS, - String.valueOf(TimeUnit.SECONDS.toMillis(10))), - HIVE_METASTORE_EXTENSION.hiveConf()); + catalog = initCatalog("hive", ImmutableMap.of()); String dbPath = HIVE_METASTORE_EXTENSION.metastore().getDatabasePath(DB_NAME); Database db = new Database(DB_NAME, "description", dbPath, Maps.newHashMap()); HIVE_METASTORE_EXTENSION.metastoreClient().createDatabase(db); } + @Override + protected HiveCatalog initCatalog(String catalogName, Map additionalProperties) { + Map properties = + ImmutableMap.of( + CatalogProperties.CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS, + String.valueOf(TimeUnit.SECONDS.toMillis(10))); + + return (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), + catalogName, + ImmutableMap.builder() + .putAll(properties) + .putAll(additionalProperties) + .build(), + HIVE_METASTORE_EXTENSION.hiveConf()); + } + @AfterEach public void cleanup() throws Exception { HIVE_METASTORE_EXTENSION.metastore().reset(); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java index 55be034221ae..dce8f7ff0f8c 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java @@ -78,7 +78,7 @@ public void setUp(NessieClientFactory clientFactory, @NessieClientUri URI nessie initialHashOfDefaultBranch = api.getDefaultBranch().getHash(); uri = nessieUri.toASCIIString(); hadoopConfig = new Configuration(); - catalog = initNessieCatalog("main"); + catalog = initCatalog("nessie", ImmutableMap.of()); } @AfterEach @@ -112,18 +112,28 @@ private void resetData() throws NessieConflictException, NessieNotFoundException .assign(); } - private NessieCatalog initNessieCatalog(String ref) { + @Override + protected NessieCatalog initCatalog( + String catalogName, Map additionalProperties) { Map options = ImmutableMap.of( "type", "nessie", "ref", - ref, + "main", CatalogProperties.URI, uri, CatalogProperties.WAREHOUSE_LOCATION, temp.toUri().toString()); - return (NessieCatalog) CatalogUtil.buildIcebergCatalog("nessie", options, hadoopConfig); + + return (NessieCatalog) + CatalogUtil.buildIcebergCatalog( + catalogName, + ImmutableMap.builder() + .putAll(options) + .putAll(additionalProperties) + .build(), + hadoopConfig); } @Override diff --git a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java index 4c4860e88a19..a709d814344f 100644 --- a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java @@ -20,6 +20,7 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.util.Map; import org.apache.iceberg.catalog.CatalogTests; import org.apache.iceberg.util.PropertyUtil; import org.junit.jupiter.api.AfterAll; @@ -63,6 +64,11 @@ protected RESTCatalog catalog() { return restCatalog; } + @Override + protected RESTCatalog initCatalog(String catalogName, Map additionalProperties) { + return RCKUtils.initCatalogClient(additionalProperties); + } + @Override protected boolean requiresNamespaceCreate() { return PropertyUtil.propertyAsBoolean( From 8e0031c01fa5da75555938dd527c690a0727d50e Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 28 Nov 2024 11:34:23 -0800 Subject: [PATCH 206/313] Spark: remove ROW_POSITION from project schema (#11610) --- .../iceberg/spark/source/BaseBatchReader.java | 17 +---------------- .../iceberg/spark/source/BaseBatchReader.java | 17 +---------------- .../iceberg/spark/source/BaseBatchReader.java | 17 +---------------- 3 files changed, 3 insertions(+), 48 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 49c43952135c..c05b694a60dc 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark.source; -import java.util.List; import java.util.Map; import java.util.Set; import org.apache.iceberg.FileFormat; @@ -32,12 +31,10 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; import org.apache.spark.sql.vectorized.ColumnarBatch; abstract class BaseBatchReader extends BaseReader { @@ -84,21 +81,9 @@ private CloseableIterable newParquetIterable( SparkDeleteFilter deleteFilter) { // get required schema if there are deletes Schema requiredSchema = deleteFilter != null ? deleteFilter.requiredSchema() : expectedSchema(); - boolean hasPositionDelete = deleteFilter != null ? deleteFilter.hasPosDeletes() : false; - Schema projectedSchema = requiredSchema; - if (hasPositionDelete) { - // We need to add MetadataColumns.ROW_POSITION in the schema for - // ReadConf.generateOffsetToStartPos(Schema schema). This is not needed any - // more after #10107 is merged. - List columns = Lists.newArrayList(requiredSchema.columns()); - if (!columns.contains(MetadataColumns.ROW_POSITION)) { - columns.add(MetadataColumns.ROW_POSITION); - projectedSchema = new Schema(columns); - } - } return Parquet.read(inputFile) - .project(projectedSchema) + .project(requiredSchema) .split(start, length) .createBatchedReaderFunc( fileSchema -> diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 49c43952135c..c05b694a60dc 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark.source; -import java.util.List; import java.util.Map; import java.util.Set; import org.apache.iceberg.FileFormat; @@ -32,12 +31,10 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; import org.apache.spark.sql.vectorized.ColumnarBatch; abstract class BaseBatchReader extends BaseReader { @@ -84,21 +81,9 @@ private CloseableIterable newParquetIterable( SparkDeleteFilter deleteFilter) { // get required schema if there are deletes Schema requiredSchema = deleteFilter != null ? deleteFilter.requiredSchema() : expectedSchema(); - boolean hasPositionDelete = deleteFilter != null ? deleteFilter.hasPosDeletes() : false; - Schema projectedSchema = requiredSchema; - if (hasPositionDelete) { - // We need to add MetadataColumns.ROW_POSITION in the schema for - // ReadConf.generateOffsetToStartPos(Schema schema). This is not needed any - // more after #10107 is merged. - List columns = Lists.newArrayList(requiredSchema.columns()); - if (!columns.contains(MetadataColumns.ROW_POSITION)) { - columns.add(MetadataColumns.ROW_POSITION); - projectedSchema = new Schema(columns); - } - } return Parquet.read(inputFile) - .project(projectedSchema) + .project(requiredSchema) .split(start, length) .createBatchedReaderFunc( fileSchema -> diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 49c43952135c..c05b694a60dc 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark.source; -import java.util.List; import java.util.Map; import java.util.Set; import org.apache.iceberg.FileFormat; @@ -32,12 +31,10 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; import org.apache.spark.sql.vectorized.ColumnarBatch; abstract class BaseBatchReader extends BaseReader { @@ -84,21 +81,9 @@ private CloseableIterable newParquetIterable( SparkDeleteFilter deleteFilter) { // get required schema if there are deletes Schema requiredSchema = deleteFilter != null ? deleteFilter.requiredSchema() : expectedSchema(); - boolean hasPositionDelete = deleteFilter != null ? deleteFilter.hasPosDeletes() : false; - Schema projectedSchema = requiredSchema; - if (hasPositionDelete) { - // We need to add MetadataColumns.ROW_POSITION in the schema for - // ReadConf.generateOffsetToStartPos(Schema schema). This is not needed any - // more after #10107 is merged. - List columns = Lists.newArrayList(requiredSchema.columns()); - if (!columns.contains(MetadataColumns.ROW_POSITION)) { - columns.add(MetadataColumns.ROW_POSITION); - projectedSchema = new Schema(columns); - } - } return Parquet.read(inputFile) - .project(projectedSchema) + .project(requiredSchema) .split(start, length) .createBatchedReaderFunc( fileSchema -> From 3a04257e49b555b6013f1b984e1c1802e53e1956 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 28 Nov 2024 20:44:49 +0100 Subject: [PATCH 207/313] Default to `overwrite` when operation is missing (#11421) * Default to `overwrite` when operation is missing * Update wording Co-authored-by: Russell Spitzer * Feedback * Spotless.. * Convert an empty summary to `null` * Modify behavior and add some tests --------- Co-authored-by: Russell Spitzer --- .../org/apache/iceberg/SnapshotParser.java | 32 +++++--- .../org/apache/iceberg/TestSnapshotJson.java | 73 +++++++++++++++++++ 2 files changed, 96 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotParser.java b/core/src/main/java/org/apache/iceberg/SnapshotParser.java index bc5ef6094695..b5ac3ec718ac 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotParser.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotParser.java @@ -31,8 +31,11 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.util.JsonUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SnapshotParser { + private static final Logger LOG = LoggerFactory.getLogger(SnapshotParser.class); private SnapshotParser() {} @@ -129,17 +132,28 @@ static Snapshot fromJson(JsonNode node) { "Cannot parse summary from non-object value: %s", sNode); - ImmutableMap.Builder builder = ImmutableMap.builder(); - Iterator fields = sNode.fieldNames(); - while (fields.hasNext()) { - String field = fields.next(); - if (field.equals(OPERATION)) { - operation = JsonUtil.getString(OPERATION, sNode); - } else { - builder.put(field, JsonUtil.getString(field, sNode)); + if (sNode.size() > 0) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + Iterator fields = sNode.fieldNames(); + while (fields.hasNext()) { + String field = fields.next(); + if (field.equals(OPERATION)) { + operation = JsonUtil.getString(OPERATION, sNode); + } else { + builder.put(field, JsonUtil.getString(field, sNode)); + } + } + summary = builder.build(); + + // When the operation is not found, default to overwrite + // to ensure that we can read the summary without raising an exception + if (operation == null) { + LOG.warn( + "Encountered invalid summary for snapshot {}: the field 'operation' is required but missing, setting 'operation' to overwrite", + snapshotId); + operation = DataOperations.OVERWRITE; } } - summary = builder.build(); } Integer schemaId = JsonUtil.getIntOrNull(SCHEMA_ID, node); diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index ee1239074997..8a067fc4dc44 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -176,4 +176,77 @@ private String createManifestListWithManifestFiles(long snapshotId, Long parentS return localInput(manifestList).location(); } + + @Test + public void testJsonConversionSummaryWithoutOperation() { + // This behavior is out of spec, but we don't want to fail on it. + // Instead, the operation will be set to overwrite, to ensure that it will produce + // correct metadata when it is written + + long currentMs = System.currentTimeMillis(); + String json = + String.format( + "{\n" + + " \"snapshot-id\" : 2,\n" + + " \"parent-snapshot-id\" : 1,\n" + + " \"timestamp-ms\" : %s,\n" + + " \"summary\" : {\n" + + " \"files-added\" : \"4\",\n" + + " \"files-deleted\" : \"100\"\n" + + " },\n" + + " \"manifests\" : [ \"/tmp/manifest1.avro\", \"/tmp/manifest2.avro\" ],\n" + + " \"schema-id\" : 3\n" + + "}", + currentMs); + + Snapshot snap = SnapshotParser.fromJson(json); + String expected = + String.format( + "{\n" + + " \"snapshot-id\" : 2,\n" + + " \"parent-snapshot-id\" : 1,\n" + + " \"timestamp-ms\" : %s,\n" + + " \"summary\" : {\n" + + " \"operation\" : \"overwrite\",\n" + + " \"files-added\" : \"4\",\n" + + " \"files-deleted\" : \"100\"\n" + + " },\n" + + " \"manifests\" : [ \"/tmp/manifest1.avro\", \"/tmp/manifest2.avro\" ],\n" + + " \"schema-id\" : 3\n" + + "}", + currentMs); + assertThat(SnapshotParser.toJson(snap)).isEqualTo(expected); + } + + @Test + public void testJsonConversionEmptySummary() { + // This behavior is out of spec, but we don't want to fail on it. + // Instead, when we find an empty summary, we'll just set it to null + + long currentMs = System.currentTimeMillis(); + String json = + String.format( + "{\n" + + " \"snapshot-id\" : 2,\n" + + " \"parent-snapshot-id\" : 1,\n" + + " \"timestamp-ms\" : %s,\n" + + " \"summary\" : { },\n" + + " \"manifests\" : [ \"/tmp/manifest1.avro\", \"/tmp/manifest2.avro\" ],\n" + + " \"schema-id\" : 3\n" + + "}", + currentMs); + + Snapshot snap = SnapshotParser.fromJson(json); + String expected = + String.format( + "{\n" + + " \"snapshot-id\" : 2,\n" + + " \"parent-snapshot-id\" : 1,\n" + + " \"timestamp-ms\" : %s,\n" + + " \"manifests\" : [ \"/tmp/manifest1.avro\", \"/tmp/manifest2.avro\" ],\n" + + " \"schema-id\" : 3\n" + + "}", + currentMs); + assertThat(SnapshotParser.toJson(snap)).isEqualTo(expected); + } } From 8fccdec9578edefc14af9563908fe37e645a2d04 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 28 Nov 2024 23:28:34 +0100 Subject: [PATCH 208/313] Core,API: Set `503: added_snapshot_id` as required (#11626) * Core,API: Set `503: added_snapshot_id` as required `503: added_snapshot_id` field should be written as a required field, but currently it is written as optional. As the reference implementation should produce metadata that is as close to the spec as possible. For reading, this isn't a problem with the current Java implementation as it will still read optional fields as required, but only thrown an error when it encounters a `null` value. * Improve constructor instead --- .../java/org/apache/iceberg/ManifestFile.java | 2 +- .../java/org/apache/iceberg/BaseSnapshot.java | 3 +- .../apache/iceberg/GenericManifestFile.java | 28 ++++++++++++++++++- .../java/org/apache/iceberg/V2Metadata.java | 2 +- .../java/org/apache/iceberg/V3Metadata.java | 2 +- .../iceberg/TestMetadataUpdateParser.java | 4 +-- .../org/apache/iceberg/TestSnapshotJson.java | 4 +-- .../org/apache/iceberg/TestTableMetadata.java | 3 +- .../iceberg/avro/TestReadProjection.java | 22 +++++++++++++++ 9 files changed, 60 insertions(+), 10 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/ManifestFile.java b/api/src/main/java/org/apache/iceberg/ManifestFile.java index 8f20697c7780..60372636e14e 100644 --- a/api/src/main/java/org/apache/iceberg/ManifestFile.java +++ b/api/src/main/java/org/apache/iceberg/ManifestFile.java @@ -49,7 +49,7 @@ public interface ManifestFile { Types.LongType.get(), "Lowest sequence number in the manifest"); Types.NestedField SNAPSHOT_ID = - optional( + required( 503, "added_snapshot_id", Types.LongType.get(), "Snapshot ID that added the manifest"); Types.NestedField ADDED_FILES_COUNT = optional(504, "added_files_count", Types.IntegerType.get(), "Added entry count"); diff --git a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java index a3c4fc8738cd..58dec570d1fb 100644 --- a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java @@ -138,7 +138,8 @@ private void cacheManifests(FileIO fileIO) { allManifests = Lists.transform( Arrays.asList(v1ManifestLocations), - location -> new GenericManifestFile(fileIO.newInputFile(location), 0)); + location -> + new GenericManifestFile(fileIO.newInputFile(location), 0, this.snapshotId)); } if (allManifests == null) { diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java index 7707c57a6905..a079f5fb7bd4 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java @@ -87,6 +87,26 @@ public GenericManifestFile(Schema avroSchema) { this.keyMetadata = null; } + GenericManifestFile(InputFile file, int specId, long snapshotId) { + super(ManifestFile.schema().columns().size()); + this.avroSchema = AVRO_SCHEMA; + this.file = file; + this.manifestPath = file.location(); + this.length = null; // lazily loaded from file + this.specId = specId; + this.sequenceNumber = 0; + this.minSequenceNumber = 0; + this.snapshotId = snapshotId; + this.addedFilesCount = null; + this.addedRowsCount = null; + this.existingFilesCount = null; + this.existingRowsCount = null; + this.deletedFilesCount = null; + this.deletedRowsCount = null; + this.partitions = null; + this.keyMetadata = null; + } + /** Adjust the arg order to avoid conflict with the public constructor below */ GenericManifestFile( String path, @@ -167,7 +187,13 @@ private GenericManifestFile(GenericManifestFile toCopy) { super(toCopy); this.avroSchema = toCopy.avroSchema; this.manifestPath = toCopy.manifestPath; - this.length = toCopy.length; + try { + this.length = toCopy.length(); + } catch (UnsupportedOperationException e) { + // Can be removed when embedded manifests are dropped + // DummyFileIO does not support .length() + this.length = null; + } this.specId = toCopy.specId; this.content = toCopy.content; this.sequenceNumber = toCopy.sequenceNumber; diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index bb715385610b..18c3b0a40613 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -40,7 +40,7 @@ private V2Metadata() {} ManifestFile.MANIFEST_CONTENT.asRequired(), ManifestFile.SEQUENCE_NUMBER.asRequired(), ManifestFile.MIN_SEQUENCE_NUMBER.asRequired(), - ManifestFile.SNAPSHOT_ID.asRequired(), + ManifestFile.SNAPSHOT_ID, ManifestFile.ADDED_FILES_COUNT.asRequired(), ManifestFile.EXISTING_FILES_COUNT.asRequired(), ManifestFile.DELETED_FILES_COUNT.asRequired(), diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java index 12f4a2058748..fc11a7df03de 100644 --- a/core/src/main/java/org/apache/iceberg/V3Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -40,7 +40,7 @@ private V3Metadata() {} ManifestFile.MANIFEST_CONTENT.asRequired(), ManifestFile.SEQUENCE_NUMBER.asRequired(), ManifestFile.MIN_SEQUENCE_NUMBER.asRequired(), - ManifestFile.SNAPSHOT_ID.asRequired(), + ManifestFile.SNAPSHOT_ID, ManifestFile.ADDED_FILES_COUNT.asRequired(), ManifestFile.EXISTING_FILES_COUNT.asRequired(), ManifestFile.DELETED_FILES_COUNT.asRequired(), diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java index cae19fece4e9..79c3761fa8c3 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java @@ -1244,8 +1244,8 @@ private String createManifestListWithManifestFiles(long snapshotId, Long parentS List manifests = ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manifest1.avro"), 0), - new GenericManifestFile(localInput("file:/tmp/manifest2.avro"), 0)); + new GenericManifestFile(localInput("file:/tmp/manifest1.avro"), 0, snapshotId), + new GenericManifestFile(localInput("file:/tmp/manifest2.avro"), 0, snapshotId)); try (ManifestListWriter writer = ManifestLists.write(1, Files.localOutput(manifestList), snapshotId, parentSnapshotId, 0)) { diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index 8a067fc4dc44..e4c2ba5ec2df 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -166,8 +166,8 @@ private String createManifestListWithManifestFiles(long snapshotId, Long parentS List manifests = ImmutableList.of( - new GenericManifestFile(localInput("file:/tmp/manifest1.avro"), 0), - new GenericManifestFile(localInput("file:/tmp/manifest2.avro"), 0)); + new GenericManifestFile(localInput("file:/tmp/manifest1.avro"), 0, snapshotId), + new GenericManifestFile(localInput("file:/tmp/manifest2.avro"), 0, snapshotId)); try (ManifestListWriter writer = ManifestLists.write(1, Files.localOutput(manifestList), snapshotId, parentSnapshotId, 0)) { diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 64c410b46427..6d066e8a654c 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -1663,7 +1663,8 @@ private String createManifestListWithManifestFile( try (ManifestListWriter writer = ManifestLists.write(1, Files.localOutput(manifestList), snapshotId, parentSnapshotId, 0)) { writer.addAll( - ImmutableList.of(new GenericManifestFile(localInput(manifestFile), SPEC_5.specId()))); + ImmutableList.of( + new GenericManifestFile(localInput(manifestFile), SPEC_5.specId(), snapshotId))); } return localInput(manifestList).location(); diff --git a/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java b/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java index ead17e9f9c42..30de81266efc 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java @@ -62,6 +62,28 @@ public void testFullProjection() throws Exception { assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); } + @Test + public void testReadOptionalAsRequired() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Record record = new Record(AvroSchemaUtil.convert(writeSchema, "table")); + record.put("id", 34L); + record.put("data", "test"); + + Schema readSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.required(1, "data", Types.StringType.get())); + + Record projected = writeAndRead("read_optional_as_required", writeSchema, readSchema, record); + + int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.get("data")); + assertThat(cmp).as("Should contain the correct data/renamed value").isEqualTo(0); + } + @Test public void testReorderedFullProjection() throws Exception { Schema schema = From 7c7b4bac9097217f5563338dbc4b30d1559b5d76 Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+sungwy@users.noreply.github.com> Date: Fri, 29 Nov 2024 04:39:24 -0500 Subject: [PATCH 209/313] Add GitHub Action to publish the `docker-rest-fixture` container (#11632) * Create publish-docker.yml * fix vars * add license * checkout java * Publish with version tag on tag push event * rename docker image * use DOCKERHUB_NAME variable in tag * fix tagged publication * fix * fix2 * move comment to related step --- .github/workflows/publish-docker.yml | 55 ++++++++++++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 .github/workflows/publish-docker.yml diff --git a/.github/workflows/publish-docker.yml b/.github/workflows/publish-docker.yml new file mode 100644 index 000000000000..7abec0861142 --- /dev/null +++ b/.github/workflows/publish-docker.yml @@ -0,0 +1,55 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +name: Build and Push Docker Image + +on: + push: + tags: + - 'apache-iceberg-[0-9]+.[0-9]+.[0-9]+' + workflow_dispatch: + +env: + DOCKER_IMAGE_TAG: iceberg-rest-fixture + DOCKER_IMAGE_VERSION: latest + +jobs: + build: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - uses: actions/setup-java@v4 + with: + distribution: zulu + java-version: 21 + - name: Build Iceberg Open API project + run: ./gradlew :iceberg-open-api:shadowJar + - name: Login to Docker Hub + run: | + docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} + - name: Set the tagged version + # for tag 'apache-iceberg-1.7.0', publish image 'apache/iceberg-rest-fixture:1.7.1' + if: github.event_name == 'push' && contains(github.ref, 'refs/tags/') + run: | + echo "DOCKER_IMAGE_VERSION=`echo ${{ github.ref }} | tr -d -c 0-9.`" >> "$GITHUB_ENV" + - name: Build Docker Image + run: docker build -t ${{ secrets.DOCKERHUB_USER }}/$DOCKER_IMAGE_TAG:$DOCKER_IMAGE_VERSION -f docker/iceberg-rest-adapter-image/Dockerfile . + - name: Push Docker Image + run: | + docker push ${{ secrets.DOCKERHUB_USER }}/$DOCKER_IMAGE_TAG:$DOCKER_IMAGE_VERSION From e770facc3e7cbccb719b3ae5263cd1ece181f9ea Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Sat, 30 Nov 2024 00:05:44 +0900 Subject: [PATCH 210/313] Core, GCS, Spark: Replace wrong order of assertion (#11677) --- core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java | 2 +- .../test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java | 2 +- .../iceberg/spark/source/TestIcebergSourceTablesBase.java | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java index cfea9740ebd9..44c21113c3d6 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java @@ -49,7 +49,7 @@ public void testFilterAndRemovePrefix() { Properties actual = JdbcUtil.filterAndRemovePrefix(input, "jdbc."); - assertThat(expected).isEqualTo(actual); + assertThat(actual).isEqualTo(expected); } @Test diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java index 6302f664b70a..37fd5e65dcd7 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSFileIOTest.java @@ -106,7 +106,7 @@ public void newInputFile() throws IOException { IOUtil.readFully(is, actual, 0, actual.length); } - assertThat(expected).isEqualTo(actual); + assertThat(actual).isEqualTo(expected); io.deleteFile(in); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 21afd7460ec6..dcd347a34f81 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -2054,9 +2054,9 @@ public void testFilesTablePartitionId() { .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) .collect(Collectors.toList()); - assertThat(ImmutableList.of(spec0, spec1)) + assertThat(actual) .as("Should have two partition specs") - .isEqualTo(actual); + .isEqualTo(ImmutableList.of(spec0, spec1)); } @Test From f978fe534447fd487a8d8d4c2d6c64404ac504f7 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Sun, 1 Dec 2024 12:42:30 +0530 Subject: [PATCH 211/313] REST: Clean up `iceberg-rest-fixture` docker image naming (#11676) * REST: Refactor docker image files * update readme --- ...ocker.yml => publish-iceberg-rest-fixture-docker.yml} | 9 +++++---- .../Dockerfile | 0 .../README.md | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) rename .github/workflows/{publish-docker.yml => publish-iceberg-rest-fixture-docker.yml} (82%) rename docker/{iceberg-rest-adapter-image => iceberg-rest-fixture}/Dockerfile (100%) rename docker/{iceberg-rest-adapter-image => iceberg-rest-fixture}/README.md (97%) diff --git a/.github/workflows/publish-docker.yml b/.github/workflows/publish-iceberg-rest-fixture-docker.yml similarity index 82% rename from .github/workflows/publish-docker.yml rename to .github/workflows/publish-iceberg-rest-fixture-docker.yml index 7abec0861142..eb946163fdff 100644 --- a/.github/workflows/publish-docker.yml +++ b/.github/workflows/publish-iceberg-rest-fixture-docker.yml @@ -17,7 +17,7 @@ # under the License. # -name: Build and Push Docker Image +name: Build and Push 'iceberg-rest-fixture' Docker Image on: push: @@ -28,6 +28,7 @@ on: env: DOCKER_IMAGE_TAG: iceberg-rest-fixture DOCKER_IMAGE_VERSION: latest + DOCKER_REPOSITORY: apache jobs: build: @@ -44,12 +45,12 @@ jobs: run: | docker login -u ${{ secrets.DOCKERHUB_USER }} -p ${{ secrets.DOCKERHUB_TOKEN }} - name: Set the tagged version - # for tag 'apache-iceberg-1.7.0', publish image 'apache/iceberg-rest-fixture:1.7.1' + # for tag 'apache-iceberg-1.7.1', publish image 'apache/iceberg-rest-fixture:1.7.1' if: github.event_name == 'push' && contains(github.ref, 'refs/tags/') run: | echo "DOCKER_IMAGE_VERSION=`echo ${{ github.ref }} | tr -d -c 0-9.`" >> "$GITHUB_ENV" - name: Build Docker Image - run: docker build -t ${{ secrets.DOCKERHUB_USER }}/$DOCKER_IMAGE_TAG:$DOCKER_IMAGE_VERSION -f docker/iceberg-rest-adapter-image/Dockerfile . + run: docker build -t $DOCKER_REPOSITORY/$DOCKER_IMAGE_TAG:$DOCKER_IMAGE_VERSION -f docker/iceberg-rest-fixture/Dockerfile . - name: Push Docker Image run: | - docker push ${{ secrets.DOCKERHUB_USER }}/$DOCKER_IMAGE_TAG:$DOCKER_IMAGE_VERSION + docker push $DOCKER_REPOSITORY/$DOCKER_IMAGE_TAG:$DOCKER_IMAGE_VERSION diff --git a/docker/iceberg-rest-adapter-image/Dockerfile b/docker/iceberg-rest-fixture/Dockerfile similarity index 100% rename from docker/iceberg-rest-adapter-image/Dockerfile rename to docker/iceberg-rest-fixture/Dockerfile diff --git a/docker/iceberg-rest-adapter-image/README.md b/docker/iceberg-rest-fixture/README.md similarity index 97% rename from docker/iceberg-rest-adapter-image/README.md rename to docker/iceberg-rest-fixture/README.md index 661b5397e241..3805cc2468cb 100644 --- a/docker/iceberg-rest-adapter-image/README.md +++ b/docker/iceberg-rest-fixture/README.md @@ -32,7 +32,7 @@ When making changes to the local files and test them out, you can build the imag ./gradlew :iceberg-open-api:shadowJar # Rebuild the docker image -docker image rm -f apache/iceberg-rest-adapter && docker build -t apache/iceberg-rest-adapter -f docker/iceberg-rest-adapter-image/Dockerfile . +docker image rm -f apache/iceberg-rest-fixture && docker build -t apache/iceberg-rest-fixture -f docker/iceberg-rest-fixture/Dockerfile . ``` ## Browse From 233364044e058799b8e1882f1a0282849ef8b077 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 06:22:29 +0100 Subject: [PATCH 212/313] Build: Bump software.amazon.awssdk:bom from 2.29.20 to 2.29.23 (#11683) Bumps software.amazon.awssdk:bom from 2.29.20 to 2.29.23. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 860e0654ea8b..5ce2d539309f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.29.20" +awssdk-bom = "2.29.23" azuresdk-bom = "1.2.29" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" From 578dda86dc49e09e9686fba3c19ee4018c9a8d7e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 06:22:49 +0100 Subject: [PATCH 213/313] Build: Bump mkdocs-material from 9.5.45 to 9.5.46 (#11680) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.45 to 9.5.46. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.5.45...9.5.46) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 7cbbaba724ca..59dcb22c7e3e 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.3.7 -mkdocs-material==9.5.45 +mkdocs-material==9.5.46 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.2 From bc36f5e33fd71335e91f04aa70a199243fd15897 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Mon, 2 Dec 2024 14:25:10 +0900 Subject: [PATCH 214/313] REST: Use `HEAD` request to check table existence (#10999) --- .../apache/iceberg/rest/CatalogHandlers.java | 7 +++ .../iceberg/rest/RESTSessionCatalog.java | 12 +++++ .../iceberg/rest/RESTCatalogAdapter.java | 8 +++ .../apache/iceberg/rest/TestRESTCatalog.java | 54 +++++++++---------- 4 files changed, 54 insertions(+), 27 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java index 76fafe48f5b6..563853e3f033 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -311,6 +311,13 @@ public static void purgeTable(Catalog catalog, TableIdentifier ident) { } } + public static void tableExists(Catalog catalog, TableIdentifier ident) { + boolean exists = catalog.tableExists(ident); + if (!exists) { + throw new NoSuchTableException("Table does not exist: %s", ident); + } + } + public static LoadTableResponse loadTable(Catalog catalog, TableIdentifier ident) { Table table = catalog.loadTable(ident); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 1bf57dd13c69..331ea0de3697 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -432,6 +432,18 @@ public void renameTable(SessionContext context, TableIdentifier from, TableIdent client.post(paths.rename(), request, null, headers(context), ErrorHandlers.tableErrorHandler()); } + @Override + public boolean tableExists(SessionContext context, TableIdentifier identifier) { + checkIdentifierIsValid(identifier); + + try { + client.head(paths.table(identifier), headers(context), ErrorHandlers.tableErrorHandler()); + return true; + } catch (NoSuchTableException e) { + return false; + } + } + private LoadTableResponse loadInternal( SessionContext context, TableIdentifier identifier, SnapshotMode mode) { Endpoint.check(endpoints, Endpoint.V1_LOAD_TABLE); diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index aa77b5ad10b6..87b693e206ae 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -137,6 +137,7 @@ enum Route { ResourcePaths.V1_TABLES, CreateTableRequest.class, LoadTableResponse.class), + TABLE_EXISTS(HTTPMethod.HEAD, ResourcePaths.V1_TABLE), LOAD_TABLE(HTTPMethod.GET, ResourcePaths.V1_TABLE, null, LoadTableResponse.class), REGISTER_TABLE( HTTPMethod.POST, @@ -392,6 +393,13 @@ public T handleRequest( return null; } + case TABLE_EXISTS: + { + TableIdentifier ident = tableIdentFromPathVars(vars); + CatalogHandlers.tableExists(catalog, ident); + return null; + } + case LOAD_TABLE: { TableIdentifier ident = tableIdentFromPathVars(vars); diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 232cfd31d1a6..973e394b30c7 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -346,11 +346,11 @@ public void testCatalogBasicBearerToken() { any()); Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(catalogHeaders), any()); } @@ -393,11 +393,11 @@ public void testCatalogCredentialNoOauth2ServerUri() { // use the catalog token for all interactions Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(catalogHeaders), any()); } @@ -448,11 +448,11 @@ public void testCatalogCredential(String oauth2ServerUri) { // use the catalog token for all interactions Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(catalogHeaders), any()); } @@ -506,14 +506,14 @@ public void testCatalogBearerTokenWithClientCredential(String oauth2ServerUri) { eq(OAuthTokenResponse.class), eq(catalogHeaders), any()); - // use the context token for table load + // use the context token for table existence check Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(contextHeaders), any()); } @@ -579,14 +579,14 @@ public void testCatalogCredentialWithClientCredential(String oauth2ServerUri) { eq(OAuthTokenResponse.class), eq(catalogHeaders), any()); - // use the context token for table load + // use the context token for table existence check Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(contextHeaders), any()); } @@ -654,14 +654,14 @@ public void testCatalogBearerTokenAndCredentialWithClientCredential(String oauth eq(OAuthTokenResponse.class), eq(catalogHeaders), any()); - // use the context token for table load + // use the context token for table existence check Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(contextHeaders), any()); } @@ -845,11 +845,11 @@ private void testClientAuth( } Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(expectedHeaders), any()); if (!optionalOAuthParams.isEmpty()) { @@ -1612,18 +1612,18 @@ public void testCatalogRefreshedTokenIsUsed(String oauth2ServerUri) { eq(catalogHeaders), any()); - // use the refreshed context token for table load + // use the refreshed context token for table existence check Map refreshedCatalogHeader = ImmutableMap.of( "Authorization", "Bearer token-exchange-token:sub=client-credentials-token:sub=catalog"); Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(refreshedCatalogHeader), any()); }); @@ -1735,11 +1735,11 @@ public void testCatalogExpiredBearerTokenIsRefreshedWithCredential(String oauth2 Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(ImmutableMap.of("Authorization", "Bearer token-exchange-token:sub=" + token)), any()); } @@ -1777,11 +1777,11 @@ public void testCatalogValidBearerTokenIsNotRefreshed() { Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(OAuth2Util.authHeaders(token)), any()); } @@ -1912,18 +1912,18 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh(String oauth eq(basicHeaders), any()); - // use the refreshed context token for table load + // use the refreshed context token for table existence check Map refreshedCatalogHeader = ImmutableMap.of( "Authorization", "Bearer token-exchange-token:sub=client-credentials-token:sub=catalog"); Mockito.verify(adapter) .execute( - eq(HTTPMethod.GET), + eq(HTTPMethod.HEAD), eq("v1/namespaces/ns/tables/table"), any(), any(), - eq(LoadTableResponse.class), + any(), eq(refreshedCatalogHeader), any()); }); From a993b799816c25cfeb126d9f99ca7d0648bbdcaf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 06:27:25 +0100 Subject: [PATCH 215/313] Build: Bump jackson-bom from 2.18.1 to 2.18.2 (#11681) Bumps `jackson-bom` from 2.18.1 to 2.18.2. Updates `com.fasterxml.jackson:jackson-bom` from 2.18.1 to 2.18.2 - [Commits](https://github.com/FasterXML/jackson-bom/compare/jackson-bom-2.18.1...jackson-bom-2.18.2) Updates `com.fasterxml.jackson.core:jackson-core` from 2.18.1 to 2.18.2 - [Commits](https://github.com/FasterXML/jackson-core/compare/jackson-core-2.18.1...jackson-core-2.18.2) Updates `com.fasterxml.jackson.core:jackson-databind` from 2.18.1 to 2.18.2 - [Commits](https://github.com/FasterXML/jackson/commits) Updates `com.fasterxml.jackson.core:jackson-annotations` from 2.18.1 to 2.18.2 - [Commits](https://github.com/FasterXML/jackson/commits) --- updated-dependencies: - dependency-name: com.fasterxml.jackson:jackson-bom dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: com.fasterxml.jackson.core:jackson-core dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: com.fasterxml.jackson.core:jackson-databind dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: com.fasterxml.jackson.core:jackson-annotations dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 5ce2d539309f..e5c14fd30e46 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -52,7 +52,7 @@ httpcomponents-httpclient5 = "5.4.1" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" immutables-value = "2.10.1" -jackson-bom = "2.18.1" +jackson-bom = "2.18.2" jackson211 = { strictly = "2.11.4"} # see rich version usage explanation above jackson212 = { strictly = "2.12.3"} jackson213 = { strictly = "2.13.4"} From bfeaaeb8181bf035367761a0c6c907055f4f00cf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 2 Dec 2024 06:27:43 +0100 Subject: [PATCH 216/313] Build: Bump org.xerial:sqlite-jdbc from 3.47.0.0 to 3.47.1.0 (#11682) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.47.0.0 to 3.47.1.0. - [Release notes](https://github.com/xerial/sqlite-jdbc/releases) - [Changelog](https://github.com/xerial/sqlite-jdbc/blob/master/CHANGELOG) - [Commits](https://github.com/xerial/sqlite-jdbc/compare/3.47.0.0...3.47.1.0) --- updated-dependencies: - dependency-name: org.xerial:sqlite-jdbc dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index e5c14fd30e46..e06f7ee39412 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -83,7 +83,7 @@ snowflake-jdbc = "3.20.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.4" spark-hive35 = "3.5.2" -sqlite-jdbc = "3.47.0.0" +sqlite-jdbc = "3.47.1.0" testcontainers = "1.20.4" tez010 = "0.10.4" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From d8326d876574241b4811017cd489f099d4b74f56 Mon Sep 17 00:00:00 2001 From: AGW Date: Tue, 3 Dec 2024 05:58:57 +0800 Subject: [PATCH 217/313] Spark 3.5: Make where clause case sensitive in rewrite data files (#11439) --- .../TestRewriteDataFilesProcedure.java | 23 +++++++++++++++++++ .../actions/RewriteDataFilesSparkAction.java | 4 ++++ 2 files changed, 27 insertions(+) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 93198825e326..3d3a105a14be 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -69,6 +69,29 @@ public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); } + @TestTemplate + public void testFilterCaseSensitivity() { + createTable(); + insertData(10); + sql("set %s = false", SQLConf.CASE_SENSITIVE().key()); + List expectedRecords = currentData(); + List output = + sql( + "CALL %s.system.rewrite_data_files(table=>'%s', where=>'C1 > 0')", + catalogName, tableIdent); + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(4); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + @TestTemplate public void testZOrderSortExpression() { List order = diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java index 4e381a7bd362..e04a0c88b4bb 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java @@ -62,6 +62,7 @@ import org.apache.iceberg.relocated.com.google.common.math.IntMath; import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.StructLikeMap; @@ -102,11 +103,13 @@ public class RewriteDataFilesSparkAction private boolean useStartingSequenceNumber; private RewriteJobOrder rewriteJobOrder; private FileRewriter rewriter = null; + private boolean caseSensitive; RewriteDataFilesSparkAction(SparkSession spark, Table table) { super(spark.cloneSession()); // Disable Adaptive Query Execution as this may change the output partitioning of our write spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false); + this.caseSensitive = SparkUtil.caseSensitive(spark); this.table = table; } @@ -198,6 +201,7 @@ StructLikeMap>> planFileGroups(long startingSnapshotId) table .newScan() .useSnapshot(startingSnapshotId) + .caseSensitive(caseSensitive) .filter(filter) .ignoreResiduals() .planFiles(); From af8e3f5a40f4f36bbe1d868146749e2341471586 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 2 Dec 2024 17:26:27 -0800 Subject: [PATCH 218/313] Spark: Remove extra columns for ColumnarBatch (#11551) --- .../org/apache/iceberg/data/DeleteFilter.java | 10 +++- .../apache/iceberg/data/DeleteReadTests.java | 2 +- .../data/vectorized/ColumnarBatchReader.java | 31 ++++++++++++ .../spark/source/TestSparkReaderDeletes.java | 49 ++++++++++++++++++- 4 files changed, 88 insertions(+), 4 deletions(-) diff --git a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java index 96ee6fd3f5ba..797e6d6408cc 100644 --- a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java +++ b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java @@ -54,6 +54,7 @@ public abstract class DeleteFilter { private final List posDeletes; private final List eqDeletes; private final Schema requiredSchema; + private final Schema expectedSchema; private final Accessor posAccessor; private final boolean hasIsDeletedColumn; private final int isDeletedColumnPosition; @@ -68,11 +69,12 @@ protected DeleteFilter( String filePath, List deletes, Schema tableSchema, - Schema requestedSchema, + Schema expectedSchema, DeleteCounter counter, boolean needRowPosCol) { this.filePath = filePath; this.counter = counter; + this.expectedSchema = expectedSchema; ImmutableList.Builder posDeleteBuilder = ImmutableList.builder(); ImmutableList.Builder eqDeleteBuilder = ImmutableList.builder(); @@ -95,7 +97,7 @@ protected DeleteFilter( this.posDeletes = posDeleteBuilder.build(); this.eqDeletes = eqDeleteBuilder.build(); this.requiredSchema = - fileProjection(tableSchema, requestedSchema, posDeletes, eqDeletes, needRowPosCol); + fileProjection(tableSchema, expectedSchema, posDeletes, eqDeletes, needRowPosCol); this.posAccessor = requiredSchema.accessorForField(MetadataColumns.ROW_POSITION.fieldId()); this.hasIsDeletedColumn = requiredSchema.findField(MetadataColumns.IS_DELETED.fieldId()) != null; @@ -124,6 +126,10 @@ public Schema requiredSchema() { return requiredSchema; } + public Schema expectedSchema() { + return expectedSchema; + } + public boolean hasPosDeletes() { return !posDeletes.isEmpty(); } diff --git a/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java b/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java index 7c2621494288..501929bbcae7 100644 --- a/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java +++ b/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java @@ -128,7 +128,7 @@ public void cleanup() throws IOException { dropTable("test2"); } - private void initDateTable() throws IOException { + protected void initDateTable() throws IOException { dropTable("test2"); this.dateTableName = "test2"; this.dateTable = createTable(dateTableName, DATE_SCHEMA, DATE_SPEC); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java index 77cb2ff771c8..c6f1fe8dfe62 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.data.vectorized; +import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -114,6 +115,7 @@ ColumnarBatch loadDataToColumnBatch() { if (hasEqDeletes()) { applyEqDelete(newColumnarBatch); + newColumnarBatch = removeExtraColumns(arrowColumnVectors, newColumnarBatch); } if (hasIsDeletedColumn && rowIdMapping != null) { @@ -257,5 +259,34 @@ void applyEqDelete(ColumnarBatch columnarBatch) { columnarBatch.setNumRows(currentRowId); } + + /** + * Removes extra columns added for processing equality delete filters that are not part of the + * final query output. + * + *

    During query execution, additional columns may be included in the schema to evaluate + * equality delete filters. For example, if the table schema contains columns C1, C2, C3, C4, + * and C5, and the query is 'SELECT C5 FROM table' while equality delete filters are applied on + * C3 and C4, the processing schema includes C5, C3, and C4. These extra columns (C3 and C4) are + * needed to identify rows to delete but are not included in the final result. + * + *

    This method removes these extra columns from the end of {@code arrowColumnVectors}, + * ensuring only the expected columns remain. + * + * @param arrowColumnVectors the array of column vectors representing query result data + * @param columnarBatch the original {@code ColumnarBatch} containing query results + * @return a new {@code ColumnarBatch} with extra columns removed, or the original batch if no + * extra columns were found + */ + ColumnarBatch removeExtraColumns( + ColumnVector[] arrowColumnVectors, ColumnarBatch columnarBatch) { + int expectedColumnSize = deletes.expectedSchema().columns().size(); + if (arrowColumnVectors.length > expectedColumnSize) { + ColumnVector[] newColumns = Arrays.copyOf(arrowColumnVectors, expectedColumnSize); + return new ColumnarBatch(newColumns, columnarBatch.numRows()); + } else { + return columnarBatch; + } + } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index d1ed1dc2b3cf..d47cf2512916 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -21,11 +21,13 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; import static org.apache.iceberg.spark.source.SparkSQLExecutionHelper.lastExecutedMetricValue; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.apache.spark.sql.types.DataTypes.IntegerType; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; +import java.time.LocalDate; import java.util.List; import java.util.Set; import org.apache.hadoop.conf.Configuration; @@ -86,6 +88,7 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnarBatch; import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -95,7 +98,6 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSparkReaderDeletes extends DeleteReadTests { - private static TestHiveMetastore metastore = null; protected static SparkSession spark = null; protected static HiveCatalog catalog = null; @@ -632,6 +634,51 @@ public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOExceptio assertThat(rowSet(tblName, tbl, "*")).hasSize(193); } + @TestTemplate + public void testEqualityDeleteWithDifferentScanAndDeleteColumns() throws IOException { + assumeThat(format).isEqualTo(FileFormat.PARQUET); + initDateTable(); + + Schema deleteRowSchema = dateTable.schema().select("dt"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = + Lists.newArrayList( + dataDelete.copy("dt", LocalDate.parse("2021-09-01")), + dataDelete.copy("dt", LocalDate.parse("2021-09-02")), + dataDelete.copy("dt", LocalDate.parse("2021-09-03"))); + + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + dateTable, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + dataDeletes.subList(0, 3), + deleteRowSchema); + + dateTable.newRowDelta().addDeletes(eqDeletes).commit(); + + CloseableIterable tasks = + TableScanUtil.planTasks( + dateTable.newScan().planFiles(), + TableProperties.METADATA_SPLIT_SIZE_DEFAULT, + TableProperties.SPLIT_LOOKBACK_DEFAULT, + TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); + + for (CombinedScanTask task : tasks) { + try (BatchDataReader reader = + new BatchDataReader( + // expected column is id, while the equality filter column is dt + dateTable, task, dateTable.schema(), dateTable.schema().select("id"), false, 7)) { + while (reader.next()) { + ColumnarBatch columnarBatch = reader.get(); + int numOfCols = columnarBatch.numCols(); + assertThat(numOfCols).as("Number of columns").isEqualTo(1); + assertThat(columnarBatch.column(0).dataType()).as("Column type").isEqualTo(IntegerType); + } + } + } + } + private static final Schema PROJECTION_SCHEMA = new Schema( required(1, "id", Types.IntegerType.get()), From 6501d29b2d46c8d57f46ad646e2daf7d8865f646 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 3 Dec 2024 18:54:47 +0100 Subject: [PATCH 219/313] Spark: Add view support to SparkSessionCatalog (#11388) * Spark: Add view support to SparkSessionCatalog * Don't replace view non-atomically --- .../iceberg/spark/extensions/TestViews.java | 149 ++++++++++------- .../org/apache/iceberg/spark/BaseCatalog.java | 5 +- .../apache/iceberg/spark/SparkCatalog.java | 3 +- .../iceberg/spark/SparkSessionCatalog.java | 151 +++++++++++++++++- .../iceberg/spark/SparkCatalogConfig.java | 6 +- 5 files changed, 254 insertions(+), 60 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index 61eafc5f4e78..a6149a0ae49e 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -21,12 +21,15 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatNoException; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; import java.util.Locale; import java.util.Random; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.IcebergBuild; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; @@ -35,6 +38,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.catalog.ViewCatalog; +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.SparkCatalogConfig; @@ -61,21 +65,30 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestViews extends ExtensionsTestBase { private static final Namespace NAMESPACE = Namespace.of("default"); + private static final String SPARK_CATALOG = "spark_catalog"; private final String tableName = "table"; @BeforeEach + @Override public void before() { super.before(); spark.conf().set("spark.sql.defaultCatalog", catalogName); sql("USE %s", catalogName); sql("CREATE NAMESPACE IF NOT EXISTS %s", NAMESPACE); - sql("CREATE TABLE %s (id INT, data STRING)", tableName); + sql( + "CREATE TABLE IF NOT EXISTS %s.%s (id INT, data STRING)%s", + NAMESPACE, tableName, catalogName.equals(SPARK_CATALOG) ? " USING iceberg" : ""); + sql("USE %s.%s", catalogName, NAMESPACE); } @AfterEach public void removeTable() { sql("USE %s", catalogName); - sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s.%s", NAMESPACE, tableName); + + // reset spark session catalog + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog"); } @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") @@ -85,6 +98,14 @@ public static Object[][] parameters() { SparkCatalogConfig.SPARK_WITH_VIEWS.catalogName(), SparkCatalogConfig.SPARK_WITH_VIEWS.implementation(), SparkCatalogConfig.SPARK_WITH_VIEWS.properties() + }, + { + SparkCatalogConfig.SPARK_SESSION_WITH_VIEWS.catalogName(), + SparkCatalogConfig.SPARK_SESSION_WITH_VIEWS.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.SPARK_SESSION_WITH_VIEWS.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + .build() } }; } @@ -502,11 +523,20 @@ public void readFromViewReferencingTempFunction() throws NoSuchTableException { assertThat(sql(sql)).hasSize(1).containsExactly(row(5.5)); + String expectedErrorMsg = + String.format("Cannot load function: %s.%s.%s", catalogName, NAMESPACE, functionName); + if (SPARK_CATALOG.equals(catalogName)) { + // spark session catalog tries to load a V1 function and has a different error msg + expectedErrorMsg = + String.format( + "[ROUTINE_NOT_FOUND] The function `%s`.`%s` cannot be found", + NAMESPACE, functionName); + } + // reading from a view that references a TEMP FUNCTION shouldn't be possible assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) .isInstanceOf(AnalysisException.class) - .hasMessageStartingWith( - String.format("Cannot load function: %s.%s.%s", catalogName, NAMESPACE, functionName)); + .hasMessageStartingWith(expectedErrorMsg); } @TestTemplate @@ -534,6 +564,9 @@ public void readFromViewWithCTE() throws NoSuchTableException { @TestTemplate public void rewriteFunctionIdentifier() { + assumeThat(catalogName) + .as("system namespace doesn't exist in SparkSessionCatalog") + .isNotEqualTo(SPARK_CATALOG); String viewName = viewName("rewriteFunctionIdentifier"); String sql = "SELECT iceberg_version() AS version"; @@ -579,6 +612,9 @@ public void builtinFunctionIdentifierNotRewritten() { @TestTemplate public void rewriteFunctionIdentifierWithNamespace() { + assumeThat(catalogName) + .as("system namespace doesn't exist in SparkSessionCatalog") + .isNotEqualTo(SPARK_CATALOG); String viewName = viewName("rewriteFunctionIdentifierWithNamespace"); String sql = "SELECT system.bucket(100, 'a') AS bucket_result, 'a' AS value"; @@ -596,8 +632,7 @@ public void rewriteFunctionIdentifierWithNamespace() { assertThatThrownBy(() -> sql(sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot resolve function") - .hasMessageContaining("`system`.`bucket`"); + .hasMessageContaining("Cannot resolve function `system`.`bucket`"); assertThat(sql("SELECT * FROM %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasSize(1) @@ -606,6 +641,9 @@ public void rewriteFunctionIdentifierWithNamespace() { @TestTemplate public void fullFunctionIdentifier() { + assumeThat(catalogName) + .as("system namespace doesn't exist in SparkSessionCatalog") + .isNotEqualTo(SPARK_CATALOG); String viewName = viewName("fullFunctionIdentifier"); String sql = String.format( @@ -754,10 +792,19 @@ public void renameViewToDifferentTargetCatalog() { .withSchema(schema(sql)) .create(); - assertThatThrownBy(() -> sql("ALTER VIEW %s RENAME TO spark_catalog.%s", viewName, renamedView)) + String targetCatalog = + catalogName.equals(SPARK_CATALOG) + ? SparkCatalogConfig.SPARK_WITH_VIEWS.catalogName() + : SPARK_CATALOG; + + assertThatThrownBy( + () -> + sql( + "ALTER VIEW %s RENAME TO %s.%s.%s", + viewName, targetCatalog, NAMESPACE, renamedView)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( - "Cannot move view between catalogs: from=spark_with_views and to=spark_catalog"); + "Cannot move view between catalogs: from=%s and to=%s", catalogName, targetCatalog); } @TestTemplate @@ -813,7 +860,9 @@ public void renameViewTargetAlreadyExistsAsTable() { .withSchema(schema(sql)) .create(); - sql("CREATE TABLE %s (id INT, data STRING)", target); + sql( + "CREATE TABLE %s.%s.%s (id INT, data STRING)%s", + catalogName, NAMESPACE, target, catalogName.equals(SPARK_CATALOG) ? " USING iceberg" : ""); assertThatThrownBy(() -> sql("ALTER VIEW %s RENAME TO %s", viewName, target)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( @@ -895,30 +944,6 @@ public void dropTempView() { assertThat(v1SessionCatalog().getTempView(tempView).isDefined()).isFalse(); } - /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ - @TestTemplate - public void dropV1View() { - String v1View = viewName("v1ViewToBeDropped"); - sql("USE spark_catalog"); - sql("CREATE NAMESPACE IF NOT EXISTS %s", NAMESPACE); - sql("CREATE TABLE %s (id INT, data STRING)", tableName); - sql("CREATE VIEW %s AS SELECT id FROM %s", v1View, tableName); - sql("USE %s", catalogName); - assertThat( - v1SessionCatalog() - .tableExists(new org.apache.spark.sql.catalyst.TableIdentifier(v1View))) - .isTrue(); - - sql("DROP VIEW spark_catalog.%s.%s", NAMESPACE, v1View); - assertThat( - v1SessionCatalog() - .tableExists(new org.apache.spark.sql.catalyst.TableIdentifier(v1View))) - .isFalse(); - - sql("USE spark_catalog"); - sql("DROP TABLE IF EXISTS %s", tableName); - } - private SessionCatalog v1SessionCatalog() { return spark.sessionState().catalogManager().v1SessionCatalog(); } @@ -1316,11 +1341,13 @@ public void createViewWithSubqueryExpressionInFilterThatIsRewritten() assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(5)); - sql("USE spark_catalog"); + if (!catalogName.equals(SPARK_CATALOG)) { + sql("USE spark_catalog"); - assertThatThrownBy(() -> sql(sql)) - .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("The table or view `%s` cannot be found", tableName)); + assertThatThrownBy(() -> sql(sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining(String.format("The table or view `%s` cannot be found", tableName)); + } // the underlying SQL in the View should be rewritten to have catalog & namespace assertThat(sql("SELECT * FROM %s.%s.%s", catalogName, NAMESPACE, viewName)) @@ -1341,11 +1368,13 @@ public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSu .hasSize(3) .containsExactly(row(3), row(3), row(3)); - sql("USE spark_catalog"); + if (!catalogName.equals(SPARK_CATALOG)) { + sql("USE spark_catalog"); - assertThatThrownBy(() -> sql(sql)) - .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("The table or view `%s` cannot be found", tableName)); + assertThatThrownBy(() -> sql(sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining(String.format("The table or view `%s` cannot be found", tableName)); + } // the underlying SQL in the View should be rewritten to have catalog & namespace assertThat(sql("SELECT * FROM %s.%s.%s", catalogName, NAMESPACE, viewName)) @@ -1370,6 +1399,7 @@ public void describeExtendedView() { sql( "CREATE VIEW %s (new_id COMMENT 'ID', new_data COMMENT 'DATA') COMMENT 'view comment' AS %s", viewName, sql); + String location = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)).location(); assertThat(sql("DESCRIBE EXTENDED %s", viewName)) .contains( row("new_id", "int", "ID"), @@ -1382,8 +1412,8 @@ public void describeExtendedView() { row( "View Properties", String.format( - "['format-version' = '1', 'location' = '/%s/%s', 'provider' = 'iceberg']", - NAMESPACE, viewName), + "['format-version' = '1', 'location' = '%s', 'provider' = 'iceberg']", + location), "")); } @@ -1441,12 +1471,15 @@ public void showViews() throws NoSuchTableException { row(NAMESPACE.toString(), v1, false), tempView); - assertThat(sql("SHOW VIEWS IN %s", catalogName)) - .contains( - row(NAMESPACE.toString(), prefixV2, false), - row(NAMESPACE.toString(), prefixV3, false), - row(NAMESPACE.toString(), v1, false), - tempView); + if (!"rest".equals(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE))) { + // REST catalog requires a namespace + assertThat(sql("SHOW VIEWS IN %s", catalogName)) + .contains( + row(NAMESPACE.toString(), prefixV2, false), + row(NAMESPACE.toString(), prefixV3, false), + row(NAMESPACE.toString(), v1, false), + tempView); + } assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, NAMESPACE)) .contains( @@ -1461,7 +1494,10 @@ public void showViews() throws NoSuchTableException { assertThat(sql("SHOW VIEWS LIKE 'non-existing'")).isEmpty(); - assertThat(sql("SHOW VIEWS IN spark_catalog.default")).contains(tempView); + if (!catalogName.equals(SPARK_CATALOG)) { + sql("CREATE NAMESPACE IF NOT EXISTS spark_catalog.%s", NAMESPACE); + assertThat(sql("SHOW VIEWS IN spark_catalog.%s", NAMESPACE)).contains(tempView); + } assertThat(sql("SHOW VIEWS IN global_temp")) .contains( @@ -1512,6 +1548,7 @@ public void showCreateSimpleView() { sql("CREATE VIEW %s AS %s", viewName, sql); + String location = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)).location(); String expected = String.format( "CREATE VIEW %s.%s.%s (\n" @@ -1519,10 +1556,10 @@ public void showCreateSimpleView() { + " data)\n" + "TBLPROPERTIES (\n" + " 'format-version' = '1',\n" - + " 'location' = '/%s/%s',\n" + + " 'location' = '%s',\n" + " 'provider' = 'iceberg')\n" + "AS\n%s\n", - catalogName, NAMESPACE, viewName, NAMESPACE, viewName, sql); + catalogName, NAMESPACE, viewName, location, sql); assertThat(sql("SHOW CREATE TABLE %s", viewName)).containsExactly(row(expected)); } @@ -1536,6 +1573,7 @@ public void showCreateComplexView() { + "COMMENT 'view comment' TBLPROPERTIES ('key1'='val1', 'key2'='val2') AS %s", viewName, sql); + String location = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)).location(); String expected = String.format( "CREATE VIEW %s.%s.%s (\n" @@ -1546,10 +1584,10 @@ public void showCreateComplexView() { + " 'format-version' = '1',\n" + " 'key1' = 'val1',\n" + " 'key2' = 'val2',\n" - + " 'location' = '/%s/%s',\n" + + " 'location' = '%s',\n" + " 'provider' = 'iceberg')\n" + "AS\n%s\n", - catalogName, NAMESPACE, viewName, NAMESPACE, viewName, sql); + catalogName, NAMESPACE, viewName, location, sql); assertThat(sql("SHOW CREATE TABLE %s", viewName)).containsExactly(row(expected)); } @@ -1897,6 +1935,7 @@ public void createViewWithRecursiveCycle() { @TestTemplate public void createViewWithRecursiveCycleToV1View() { + assumeThat(catalogName).isNotEqualTo(SPARK_CATALOG); String viewOne = viewName("view_one"); String viewTwo = viewName("view_two"); @@ -1908,7 +1947,7 @@ public void createViewWithRecursiveCycleToV1View() { sql("USE %s", catalogName); // viewOne points to viewTwo points to viewOne, creating a recursive cycle String view1 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewOne); - String view2 = String.format("%s.%s.%s", "spark_catalog", NAMESPACE, viewTwo); + String view2 = String.format("%s.%s.%s", SPARK_CATALOG, NAMESPACE, viewTwo); String cycle = String.format("%s -> %s -> %s", view1, view2, view1); assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS SELECT * FROM %s", viewOne, view2)) .isInstanceOf(AnalysisException.class) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java index 2082c0584608..78533d163d83 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java @@ -26,6 +26,7 @@ import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.StagingTableCatalog; import org.apache.spark.sql.connector.catalog.SupportsNamespaces; +import org.apache.spark.sql.connector.catalog.ViewCatalog; import org.apache.spark.sql.connector.iceberg.catalog.Procedure; import org.apache.spark.sql.connector.iceberg.catalog.ProcedureCatalog; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -35,7 +36,9 @@ abstract class BaseCatalog ProcedureCatalog, SupportsNamespaces, HasIcebergCatalog, - SupportsFunctions { + SupportsFunctions, + ViewCatalog, + SupportsReplaceView { private static final String USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS = "use-nullable-query-schema"; private static final boolean USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS_DEFAULT = true; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 0c361598623e..bb8926d34a96 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -120,8 +120,7 @@ * *

    */ -public class SparkCatalog extends BaseCatalog - implements org.apache.spark.sql.connector.catalog.ViewCatalog, SupportsReplaceView { +public class SparkCatalog extends BaseCatalog { private static final Set DEFAULT_NS_KEYS = ImmutableSet.of(TableCatalog.PROP_OWNER); private static final Splitter COMMA = Splitter.on(","); private static final Joiner COMMA_JOINER = Joiner.on(","); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java index fa3f1fbe4b2a..4137937ecfc6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java @@ -31,8 +31,10 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.NoSuchViewException; import org.apache.spark.sql.catalyst.analysis.NonEmptyNamespaceException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.catalyst.analysis.ViewAlreadyExistsException; import org.apache.spark.sql.connector.catalog.CatalogExtension; import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.connector.catalog.FunctionCatalog; @@ -44,6 +46,9 @@ import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.catalog.TableChange; +import org.apache.spark.sql.connector.catalog.View; +import org.apache.spark.sql.connector.catalog.ViewCatalog; +import org.apache.spark.sql.connector.catalog.ViewChange; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; @@ -55,13 +60,15 @@ * @param CatalogPlugin class to avoid casting to TableCatalog, FunctionCatalog and * SupportsNamespaces. */ -public class SparkSessionCatalog +public class SparkSessionCatalog< + T extends TableCatalog & FunctionCatalog & SupportsNamespaces & ViewCatalog> extends BaseCatalog implements CatalogExtension { private static final String[] DEFAULT_NAMESPACE = new String[] {"default"}; private String catalogName = null; private TableCatalog icebergCatalog = null; private StagingTableCatalog asStagingCatalog = null; + private ViewCatalog asViewCatalog = null; private T sessionCatalog = null; private boolean createParquetAsIceberg = false; private boolean createAvroAsIceberg = false; @@ -319,6 +326,10 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { this.asStagingCatalog = (StagingTableCatalog) icebergCatalog; } + if (icebergCatalog instanceof ViewCatalog) { + this.asViewCatalog = (ViewCatalog) icebergCatalog; + } + this.createParquetAsIceberg = options.getBoolean("parquet-enabled", createParquetAsIceberg); this.createAvroAsIceberg = options.getBoolean("avro-enabled", createAvroAsIceberg); this.createOrcAsIceberg = options.getBoolean("orc-enabled", createOrcAsIceberg); @@ -389,6 +400,10 @@ public Catalog icebergCatalog() { return ((HasIcebergCatalog) icebergCatalog).icebergCatalog(); } + private boolean isViewCatalog() { + return getSessionCatalog() instanceof ViewCatalog; + } + @Override public UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionException { try { @@ -397,4 +412,138 @@ public UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionExcep return getSessionCatalog().loadFunction(ident); } } + + @Override + public Identifier[] listViews(String... namespace) { + try { + if (null != asViewCatalog) { + return asViewCatalog.listViews(namespace); + } else if (isViewCatalog()) { + getSessionCatalog().listViews(namespace); + } + } catch (NoSuchNamespaceException e) { + throw new RuntimeException(e); + } + + return new Identifier[0]; + } + + @Override + public View loadView(Identifier ident) throws NoSuchViewException { + if (null != asViewCatalog && asViewCatalog.viewExists(ident)) { + return asViewCatalog.loadView(ident); + } else if (isViewCatalog() && getSessionCatalog().viewExists(ident)) { + return getSessionCatalog().loadView(ident); + } + + throw new NoSuchViewException(ident); + } + + @Override + public View createView( + Identifier ident, + String sql, + String currentCatalog, + String[] currentNamespace, + StructType schema, + String[] queryColumnNames, + String[] columnAliases, + String[] columnComments, + Map properties) + throws ViewAlreadyExistsException, NoSuchNamespaceException { + if (null != asViewCatalog) { + return asViewCatalog.createView( + ident, + sql, + currentCatalog, + currentNamespace, + schema, + queryColumnNames, + columnAliases, + columnComments, + properties); + } else if (isViewCatalog()) { + return getSessionCatalog() + .createView( + ident, + sql, + currentCatalog, + currentNamespace, + schema, + queryColumnNames, + columnAliases, + columnComments, + properties); + } + + throw new UnsupportedOperationException( + "Creating a view is not supported by catalog: " + catalogName); + } + + @Override + public View replaceView( + Identifier ident, + String sql, + String currentCatalog, + String[] currentNamespace, + StructType schema, + String[] queryColumnNames, + String[] columnAliases, + String[] columnComments, + Map properties) + throws NoSuchNamespaceException, NoSuchViewException { + if (asViewCatalog instanceof SupportsReplaceView) { + return ((SupportsReplaceView) asViewCatalog) + .replaceView( + ident, + sql, + currentCatalog, + currentNamespace, + schema, + queryColumnNames, + columnAliases, + columnComments, + properties); + } + + throw new UnsupportedOperationException( + "Replacing a view is not supported by catalog: " + catalogName); + } + + @Override + public View alterView(Identifier ident, ViewChange... changes) + throws NoSuchViewException, IllegalArgumentException { + if (null != asViewCatalog && asViewCatalog.viewExists(ident)) { + return asViewCatalog.alterView(ident, changes); + } else if (isViewCatalog()) { + return getSessionCatalog().alterView(ident, changes); + } + + throw new UnsupportedOperationException( + "Altering a view is not supported by catalog: " + catalogName); + } + + @Override + public boolean dropView(Identifier ident) { + if (null != asViewCatalog && asViewCatalog.viewExists(ident)) { + return asViewCatalog.dropView(ident); + } else if (isViewCatalog()) { + return getSessionCatalog().dropView(ident); + } + + return false; + } + + @Override + public void renameView(Identifier fromIdentifier, Identifier toIdentifier) + throws NoSuchViewException, ViewAlreadyExistsException { + if (null != asViewCatalog && asViewCatalog.viewExists(fromIdentifier)) { + asViewCatalog.renameView(fromIdentifier, toIdentifier); + } else if (isViewCatalog()) { + getSessionCatalog().renameView(fromIdentifier, toIdentifier); + } else { + throw new UnsupportedOperationException( + "Renaming a view is not supported by catalog: " + catalogName); + } + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java index e03f500dc46c..ef6c49db57a2 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java @@ -57,7 +57,11 @@ public enum SparkCatalogConfig { "default-namespace", "default", "cache-enabled", - "false")); + "false")), + SPARK_SESSION_WITH_VIEWS( + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of("type", "rest", "default-namespace", "default", "cache-enabled", "false")); private final String catalogName; private final String implementation; From 15bf9ca54f4fe3ef665e2be641e6fe6f28a995d3 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Wed, 4 Dec 2024 20:09:22 +0900 Subject: [PATCH 220/313] Core: Fix warning message for deprecated OAuth2 server URI (#11694) --- .../main/java/org/apache/iceberg/rest/RESTSessionCatalog.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 331ea0de3697..5c6fc49984a5 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -228,12 +228,12 @@ public void initialize(String name, Map unresolved) { && (hasInitToken || hasCredential) && !PropertyUtil.propertyAsBoolean(props, "rest.sigv4-enabled", false)) { LOG.warn( - "Iceberg REST client is missing the OAuth2 server URI configuration and defaults to {}{}. " + "Iceberg REST client is missing the OAuth2 server URI configuration and defaults to {}/{}. " + "This automatic fallback will be removed in a future Iceberg release." + "It is recommended to configure the OAuth2 endpoint using the '{}' property to be prepared. " + "This warning will disappear if the OAuth2 endpoint is explicitly configured. " + "See https://github.com/apache/iceberg/issues/10537", - props.get(CatalogProperties.URI), + RESTUtil.stripTrailingSlash(props.get(CatalogProperties.URI)), ResourcePaths.tokens(), OAuth2Properties.OAUTH2_SERVER_URI); } From c7cef9bd81d150108a56a848b7482f179ea878b8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 4 Dec 2024 12:10:24 +0100 Subject: [PATCH 221/313] Build: Bump Parquet to 1.15.0 (#11656) --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index e06f7ee39412..1bf625cbf8bb 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -75,7 +75,7 @@ netty-buffer = "4.1.115.Final" netty-buffer-compat = "4.1.115.Final" object-client-bundle = "3.3.2" orc = "1.9.5" -parquet = "1.14.4" +parquet = "1.15.0" roaringbitmap = "1.3.0" scala-collection-compat = "2.12.0" slf4j = "2.0.16" From 3278b6990c08e277b397d675f4610da395debc77 Mon Sep 17 00:00:00 2001 From: AGW Date: Wed, 4 Dec 2024 22:39:10 +0800 Subject: [PATCH 222/313] Spark 3.3, 3.4: Make where clause case sensitive in rewrite data files (#11696) --- .../TestRewriteDataFilesProcedure.java | 24 +++++++++++++++++++ .../actions/RewriteDataFilesSparkAction.java | 4 ++++ .../TestRewriteDataFilesProcedure.java | 23 ++++++++++++++++++ .../actions/RewriteDataFilesSparkAction.java | 4 ++++ 4 files changed, 55 insertions(+) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index fd8754b30d5b..f1d4a9e733f8 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -41,6 +41,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.internal.SQLConf; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -61,6 +62,29 @@ public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); } + @Test + public void testFilterCaseSensitivity() { + createTable(); + insertData(10); + sql("set %s = false", SQLConf.CASE_SENSITIVE().key()); + List expectedRecords = currentData(); + List output = + sql( + "CALL %s.system.rewrite_data_files(table=>'%s', where=>'C1 > 0')", + catalogName, tableIdent); + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(3); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + @Test public void testZOrderSortExpression() { List order = diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java index eed0b2b67b0a..73aa54ffc8a8 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java @@ -59,6 +59,7 @@ import org.apache.iceberg.relocated.com.google.common.math.IntMath; import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.StructLikeMap; @@ -94,11 +95,13 @@ public class RewriteDataFilesSparkAction private boolean useStartingSequenceNumber; private RewriteJobOrder rewriteJobOrder; private FileRewriter rewriter = null; + private boolean caseSensitive; RewriteDataFilesSparkAction(SparkSession spark, Table table) { super(spark.cloneSession()); // Disable Adaptive Query Execution as this may change the output partitioning of our write spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false); + this.caseSensitive = SparkUtil.caseSensitive(spark); this.table = table; } @@ -183,6 +186,7 @@ StructLikeMap>> planFileGroups(long startingSnapshotId) table .newScan() .useSnapshot(startingSnapshotId) + .caseSensitive(caseSensitive) .filter(filter) .ignoreResiduals() .planFiles(); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index e637950ae5d4..7c739fc8f61d 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -70,6 +70,29 @@ public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); } + @Test + public void testFilterCaseSensitivity() { + createTable(); + insertData(10); + sql("set %s = false", SQLConf.CASE_SENSITIVE().key()); + List expectedRecords = currentData(); + List output = + sql( + "CALL %s.system.rewrite_data_files(table=>'%s', where=>'C1 > 0')", + catalogName, tableIdent); + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(4); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + @Test public void testZOrderSortExpression() { List order = diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java index 0b2bbb3dfc39..ce0808da50b8 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java @@ -62,6 +62,7 @@ import org.apache.iceberg.relocated.com.google.common.math.IntMath; import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.StructLikeMap; @@ -100,11 +101,13 @@ public class RewriteDataFilesSparkAction private boolean useStartingSequenceNumber; private RewriteJobOrder rewriteJobOrder; private FileRewriter rewriter = null; + private boolean caseSensitive; RewriteDataFilesSparkAction(SparkSession spark, Table table) { super(spark.cloneSession()); // Disable Adaptive Query Execution as this may change the output partitioning of our write spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false); + this.caseSensitive = SparkUtil.caseSensitive(spark); this.table = table; } @@ -195,6 +198,7 @@ StructLikeMap>> planFileGroups(long startingSnapshotId) table .newScan() .useSnapshot(startingSnapshotId) + .caseSensitive(caseSensitive) .filter(filter) .ignoreResiduals() .planFiles(); From 8c04bcb876d46e953ece5b2951c9bd2f361783df Mon Sep 17 00:00:00 2001 From: Shohei Okumiya Date: Thu, 5 Dec 2024 00:41:10 +0900 Subject: [PATCH 223/313] Core: Generalize Util.blockLocations (#11053) --- .../src/main/java/org/apache/iceberg/hadoop/Util.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/hadoop/Util.java b/core/src/main/java/org/apache/iceberg/hadoop/Util.java index e7b6f299a2a1..806d3c91f408 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/Util.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/Util.java @@ -59,9 +59,18 @@ public static FileSystem getFs(Path path, Configuration conf) { } } + /** + * @deprecated since 1.8.0, will be removed in 1.9.0; use {@link + * Util#blockLocations(ScanTaskGroup, Configuration)} instead. + */ + @Deprecated public static String[] blockLocations(CombinedScanTask task, Configuration conf) { + return blockLocations((ScanTaskGroup) task, conf); + } + + public static String[] blockLocations(ScanTaskGroup taskGroup, Configuration conf) { Set locationSets = Sets.newHashSet(); - for (FileScanTask f : task.files()) { + for (FileScanTask f : taskGroup.tasks()) { Path path = new Path(f.file().location()); try { FileSystem fs = path.getFileSystem(conf); From 36140b819c60743b39176e28a63ab588df445329 Mon Sep 17 00:00:00 2001 From: Karuppayya Date: Wed, 4 Dec 2024 14:34:52 -0800 Subject: [PATCH 224/313] Docs: Spark procedure for stats collection (#11606) --- docs/docs/spark-procedures.md | 37 +++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index 3dc3926b92e4..c5f307a54c98 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -936,3 +936,40 @@ as an `UPDATE_AFTER` image, resulting in the following pre/post update images: |-----|--------|--------------| | 3 | Robert | UPDATE_BEFORE| | 3 | Dan | UPDATE_AFTER | + +## Table Statistics + +### `compute_table_stats` + +This procedure calculates the [Number of Distinct Values (NDV) statistics](../../format/puffin-spec.md) for a specific table. +By default, statistics are computed for all columns using the table's current snapshot. +The procedure can be optionally configured to compute statistics for a specific snapshot and/or a subset of columns. + +| Argument Name | Required? | Type | Description | +|---------------|-----------|---------------|-------------------------------------| +| `table` | ✔️ | string | Name of the table | +| `snapshot_id` | | string | Id of the snapshot to collect stats | +| `columns` | | array | Columns to collect stats | + +#### Output + +| Output Name | Type | Description | +|-------------------|--------|-------------------------------------------------| +| `statistics_file` | string | Path to stats file created from by this command | + +#### Examples + +Collect statistics of the latest snapshot of table `my_table` +```sql +CALL catalog_name.system.compute_table_stats('my_table'); +``` + +Collect statistics of the snapshot with id `snap1` of table `my_table` +```sql +CALL catalog_name.system.compute_table_stats(table => 'my_table', snapshot_id => 'snap1' ); +``` + +Collect statistics of the snapshot with id `snap1` of table `my_table` for columns `col1` and `col2` +```sql +CALL catalog_name.system.compute_table_stats(table => 'my_table', snapshot_id => 'snap1', columns => array('col1', 'col2')); +``` \ No newline at end of file From 38c8daa4eae8a75ab46571f1efce1609100f53dd Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 5 Dec 2024 18:18:10 -0800 Subject: [PATCH 225/313] Spark 3.5: Align RewritePositionDeleteFilesSparkAction filter with Spark case sensitivity (#11700) --- .../RewritePositionDeleteFilesSparkAction.java | 5 ++++- .../TestRewritePositionDeleteFilesAction.java | 18 +++++++++++++++++- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java index 282222ae716f..2562c74eafcc 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java @@ -59,6 +59,7 @@ import org.apache.iceberg.relocated.com.google.common.math.IntMath; import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.PartitionUtil; import org.apache.iceberg.util.PropertyUtil; @@ -92,11 +93,13 @@ public class RewritePositionDeleteFilesSparkAction private int maxCommits; private boolean partialProgressEnabled; private RewriteJobOrder rewriteJobOrder; + private boolean caseSensitive; RewritePositionDeleteFilesSparkAction(SparkSession spark, Table table) { super(spark); this.table = table; this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table); + this.caseSensitive = SparkUtil.caseSensitive(spark); } @Override @@ -158,7 +161,7 @@ private StructLikeMap>> planFileGroups() { private CloseableIterable planFiles(Table deletesTable) { PositionDeletesBatchScan scan = (PositionDeletesBatchScan) deletesTable.newBatchScan(); return CloseableIterable.transform( - scan.baseTableFilter(filter).ignoreResiduals().planFiles(), + scan.baseTableFilter(filter).caseSensitive(caseSensitive).ignoreResiduals().planFiles(), task -> (PositionDeletesScanTask) task); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 8547f9753f5e..12b104fca27c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.spark.sql.functions.expr; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; import java.io.File; @@ -78,6 +79,7 @@ import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -261,7 +263,8 @@ public void testRewriteFilter() throws Exception { Expression filter = Expressions.and( Expressions.greaterThan("c3", "0"), // should have no effect - Expressions.or(Expressions.equal("c1", 1), Expressions.equal("c1", 2))); + // "C1" should work because Spark defaults case sensitivity to false. + Expressions.or(Expressions.equal("C1", 1), Expressions.equal("C1", 2))); Result result = SparkActions.get(spark) @@ -283,6 +286,19 @@ public void testRewriteFilter() throws Exception { List actualDeletes = deleteRecords(table); assertEquals("Rows must match", expectedRecords, actualRecords); assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + + withSQLConf( + ImmutableMap.of(SQLConf.CASE_SENSITIVE().key(), "true"), + () -> { + assertThatThrownBy( + () -> + SparkActions.get(spark) + .rewritePositionDeletes(table) + .filter(filter) + .execute()) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot find field 'C1' in struct"); + }); } @TestTemplate From c91d3b764aff67d54ecfa942326bc2ed15a0f0bd Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Fri, 6 Dec 2024 08:46:57 -0800 Subject: [PATCH 226/313] Spark 3.5: Write DVs in Spark for V3 tables (#11561) --- .../iceberg/io/PartitioningDVWriter.java | 65 +++++++++++++ .../SparkRowLevelOperationsTestBase.java | 77 ++++++++++++++-- .../iceberg/spark/extensions/TestDelete.java | 10 +- .../iceberg/spark/extensions/TestMerge.java | 1 + .../extensions/TestMergeOnReadDelete.java | 67 ++++++++++++-- .../extensions/TestMergeOnReadMerge.java | 91 +++++++++++++++++-- .../extensions/TestMergeOnReadUpdate.java | 71 +++++++++++++-- .../iceberg/spark/extensions/TestUpdate.java | 6 ++ .../apache/iceberg/spark/SparkWriteConf.java | 7 ++ .../spark/source/SparkBatchQueryScan.java | 15 ++- .../spark/source/SparkPositionDeltaWrite.java | 46 +++++++--- .../iceberg/spark/data/TestHelpers.java | 11 +++ 12 files changed, 415 insertions(+), 52 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/io/PartitioningDVWriter.java diff --git a/core/src/main/java/org/apache/iceberg/io/PartitioningDVWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitioningDVWriter.java new file mode 100644 index 000000000000..dd338bea0ee9 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/PartitioningDVWriter.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.io; + +import java.io.IOException; +import java.util.function.Function; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * PartitioningDVWriter is a PartitioningWriter implementation that accumulates deleted positions + * for data files across different partitions and writes out deletion vector files. + */ +public class PartitioningDVWriter + implements PartitioningWriter, DeleteWriteResult> { + + private final DVFileWriter writer; + private DeleteWriteResult result; + + public PartitioningDVWriter( + OutputFileFactory fileFactory, + Function loadPreviousDeletes) { + this.writer = new BaseDVFileWriter(fileFactory, loadPreviousDeletes::apply); + } + + @Override + public void write(PositionDelete row, PartitionSpec spec, StructLike partition) { + writer.delete(row.path().toString(), row.pos(), spec, partition); + } + + @Override + public DeleteWriteResult result() { + Preconditions.checkState(result != null, "Cannot get result from unclosed writer"); + return result; + } + + @Override + public void close() throws IOException { + if (result == null) { + writer.close(); + this.result = writer.result(); + } + } +} diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 7af9dfc58737..86f754145f19 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -23,12 +23,15 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; import static org.apache.iceberg.SnapshotSummary.ADDED_DELETE_FILES_PROP; +import static org.apache.iceberg.SnapshotSummary.ADDED_DVS_PROP; import static org.apache.iceberg.SnapshotSummary.ADDED_FILES_PROP; +import static org.apache.iceberg.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; import static org.apache.iceberg.SnapshotSummary.CHANGED_PARTITION_COUNT_PROP; import static org.apache.iceberg.SnapshotSummary.DELETED_FILES_PROP; import static org.apache.iceberg.TableProperties.DATA_PLANNING_MODE; import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; import static org.apache.iceberg.TableProperties.DELETE_PLANNING_MODE; +import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.ORC_VECTORIZATION_ENABLED; import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; import static org.apache.iceberg.TableProperties.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED; @@ -59,8 +62,10 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.parquet.Parquet; @@ -98,11 +103,14 @@ public abstract class SparkRowLevelOperationsTestBase extends ExtensionsTestBase @Parameter(index = 8) protected PlanningMode planningMode; + @Parameter(index = 9) + protected int formatVersion; + @Parameters( name = "catalogName = {0}, implementation = {1}, config = {2}," + " format = {3}, vectorized = {4}, distributionMode = {5}," - + " fanout = {6}, branch = {7}, planningMode = {8}") + + " fanout = {6}, branch = {7}, planningMode = {8}, formatVersion = {9}") public static Object[][] parameters() { return new Object[][] { { @@ -116,7 +124,8 @@ public static Object[][] parameters() { WRITE_DISTRIBUTION_MODE_NONE, true, SnapshotRef.MAIN_BRANCH, - LOCAL + LOCAL, + 2 }, { "testhive", @@ -129,7 +138,8 @@ public static Object[][] parameters() { WRITE_DISTRIBUTION_MODE_NONE, false, "test", - DISTRIBUTED + DISTRIBUTED, + 2 }, { "testhadoop", @@ -140,7 +150,8 @@ public static Object[][] parameters() { WRITE_DISTRIBUTION_MODE_HASH, true, null, - LOCAL + LOCAL, + 2 }, { "spark_catalog", @@ -158,8 +169,44 @@ public static Object[][] parameters() { WRITE_DISTRIBUTION_MODE_RANGE, false, "test", - DISTRIBUTED - } + DISTRIBUTED, + 2 + }, + { + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hadoop"), + FileFormat.PARQUET, + RANDOM.nextBoolean(), + WRITE_DISTRIBUTION_MODE_HASH, + true, + null, + LOCAL, + 3 + }, + { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + "clients", + "1", + "parquet-enabled", + "false", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ), + FileFormat.AVRO, + false, + WRITE_DISTRIBUTION_MODE_RANGE, + false, + "test", + DISTRIBUTED, + 3 + }, }; } @@ -167,7 +214,7 @@ public static Object[][] parameters() { protected void initTable() { sql( - "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s')", + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s')", tableName, DEFAULT_FILE_FORMAT, fileFormat, @@ -178,7 +225,9 @@ protected void initTable() { DATA_PLANNING_MODE, planningMode.modeName(), DELETE_PLANNING_MODE, - planningMode.modeName()); + planningMode.modeName(), + FORMAT_VERSION, + formatVersion); switch (fileFormat) { case PARQUET: @@ -303,6 +352,10 @@ protected void validateSnapshot( validateProperty(snapshot, DELETED_FILES_PROP, deletedDataFiles); validateProperty(snapshot, ADDED_DELETE_FILES_PROP, addedDeleteFiles); validateProperty(snapshot, ADDED_FILES_PROP, addedDataFiles); + if (formatVersion >= 3) { + validateProperty(snapshot, ADDED_DVS_PROP, addedDeleteFiles); + assertThat(snapshot.summary()).doesNotContainKey(ADD_POS_DELETE_FILES_PROP); + } } protected void validateProperty(Snapshot snapshot, String property, Set expectedValues) { @@ -397,4 +450,12 @@ protected void assertAllBatchScansVectorized(SparkPlan plan) { List batchScans = SparkPlanUtil.collectBatchScans(plan); assertThat(batchScans).hasSizeGreaterThan(0).allMatch(SparkPlan::supportsColumnar); } + + protected void createTableWithDeleteGranularity( + String schema, String partitionedBy, DeleteGranularity deleteGranularity) { + createAndInitTable(schema, partitionedBy, null /* empty */); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); + } } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index 42eb2af774e9..c39ef335ea39 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -20,6 +20,8 @@ import static org.apache.iceberg.DataOperations.DELETE; import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.apache.iceberg.SnapshotSummary.ADDED_DVS_PROP; import static org.apache.iceberg.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; import static org.apache.iceberg.TableProperties.DELETE_DISTRIBUTION_MODE; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; @@ -187,6 +189,9 @@ public void testCoalesceDelete() throws Exception { // AQE detects that all shuffle blocks are small and processes them in 1 task // otherwise, there would be 200 tasks writing to the table validateProperty(snapshot, SnapshotSummary.ADDED_FILES_PROP, "1"); + } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { + validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); + validateProperty(snapshot, SnapshotSummary.ADDED_DVS_PROP, "4"); } else { // MoR DELETE requests the deleted records to be range distributed by partition and `_file` // each task contains only 1 file and therefore writes only 1 shuffle block @@ -521,7 +526,8 @@ public void deleteSingleRecordProducesDeleteOperation() throws NoSuchTableExcept } else { // this is a RowDelta that produces a "delete" instead of "overwrite" validateMergeOnRead(currentSnapshot, "1", "1", null); - validateProperty(currentSnapshot, ADD_POS_DELETE_FILES_PROP, "1"); + String property = formatVersion >= 3 ? ADDED_DVS_PROP : ADD_POS_DELETE_FILES_PROP; + validateProperty(currentSnapshot, property, "1"); } assertThat(sql("SELECT * FROM %s", tableName)) @@ -1292,6 +1298,8 @@ public void testDeleteWithMultipleSpecs() { Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { validateCopyOnWrite(currentSnapshot, "3", "4", "1"); + } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { + validateMergeOnRead(currentSnapshot, "3", "4", null); } else { validateMergeOnRead(currentSnapshot, "3", "3", null); } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java index a26707ef38aa..96966a2bea6f 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -231,6 +231,7 @@ public void testMergeWithVectorizedReads() { @TestTemplate public void testCoalesceMerge() { + assumeThat(formatVersion).isLessThan(3); createAndInitTable("id INT, salary INT, dep STRING"); String[] records = new String[100]; diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index 60941b8d5560..505b88711371 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -20,6 +20,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -27,6 +28,9 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; @@ -39,9 +43,11 @@ 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.SparkSQLProperties; +import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.iceberg.spark.source.SparkTable; import org.apache.iceberg.spark.source.TestSparkCatalog; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; @@ -56,10 +62,7 @@ public class TestMergeOnReadDelete extends TestDelete { @Override protected Map extraTableProperties() { return ImmutableMap.of( - TableProperties.FORMAT_VERSION, - "2", - TableProperties.DELETE_MODE, - RowLevelOperationMode.MERGE_ON_READ.modeName()); + TableProperties.DELETE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName()); } @BeforeEach @@ -93,11 +96,13 @@ public void testDeleteWithExecutorCacheLocality() throws NoSuchTableException { @TestTemplate public void testDeleteFileGranularity() throws NoSuchTableException { + assumeThat(formatVersion).isEqualTo(2); checkDeleteFileGranularity(DeleteGranularity.FILE); } @TestTemplate public void testDeletePartitionGranularity() throws NoSuchTableException { + assumeThat(formatVersion).isEqualTo(2); checkDeleteFileGranularity(DeleteGranularity.PARTITION); } @@ -182,13 +187,57 @@ public void testUnpartitionedPositionDeletesAreMaintainedDuringDelete() sql("SELECT * FROM %s ORDER BY id ASC", selectTarget())); } - private void checkDeleteFileGranularity(DeleteGranularity deleteGranularity) - throws NoSuchTableException { - createAndInitPartitionedTable(); + @TestTemplate + public void testDeleteWithDVAndHistoricalPositionDeletes() { + assumeThat(formatVersion).isEqualTo(2); + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", DeleteGranularity.PARTITION); + createBranchIfNeeded(); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }\n" + + "{ \"id\": 3, \"dep\": \"hr\" }"); + append( + commitTarget(), + "{ \"id\": 4, \"dep\": \"hr\" }\n" + + "{ \"id\": 5, \"dep\": \"hr\" }\n" + + "{ \"id\": 6, \"dep\": \"hr\" }"); + + // Produce partition scoped deletes for the two modified files + sql("DELETE FROM %s WHERE id = 1 or id = 4", commitTarget()); + // Produce 1 file-scoped deletes for the second update + Map fileGranularityProps = + ImmutableMap.of(TableProperties.DELETE_GRANULARITY, DeleteGranularity.FILE.toString()); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(fileGranularityProps)); + sql("DELETE FROM %s WHERE id = 5", commitTarget()); + + // Produce a DV which will contain 3 positions from the second data file + // 2 existing deleted positions from the earlier file-scoped and partition-scoped deletes + // and 1 new deleted position + Map updateFormatProperties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, "3"); sql( - "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(updateFormatProperties)); + sql("DELETE FROM %s where id = 6", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Set deleteFiles = + TestHelpers.deleteFiles(table, SnapshotUtil.latestSnapshot(table, branch)); + List dvs = + deleteFiles.stream().filter(ContentFileUtil::isDV).collect(Collectors.toList()); + assertThat(dvs).hasSize(1); + assertThat(dvs).allMatch(dv -> dv.recordCount() == 3); + } + + private void checkDeleteFileGranularity(DeleteGranularity deleteGranularity) + throws NoSuchTableException { + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", deleteGranularity); append(tableName, new Employee(1, "hr"), new Employee(2, "hr")); append(tableName, new Employee(3, "hr"), new Employee(4, "hr")); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java index 71ca3421f28d..7af128bcc171 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java @@ -19,8 +19,14 @@ package org.apache.iceberg.spark.extensions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -28,6 +34,8 @@ import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.Encoders; import org.junit.jupiter.api.TestTemplate; @@ -37,28 +45,95 @@ public class TestMergeOnReadMerge extends TestMerge { @Override protected Map extraTableProperties() { return ImmutableMap.of( - TableProperties.FORMAT_VERSION, - "2", - TableProperties.MERGE_MODE, - RowLevelOperationMode.MERGE_ON_READ.modeName()); + TableProperties.MERGE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName()); } @TestTemplate public void testMergeDeleteFileGranularity() { + assumeThat(formatVersion).isEqualTo(2); checkMergeDeleteGranularity(DeleteGranularity.FILE); } @TestTemplate public void testMergeDeletePartitionGranularity() { + assumeThat(formatVersion).isEqualTo(2); checkMergeDeleteGranularity(DeleteGranularity.PARTITION); } - private void checkMergeDeleteGranularity(DeleteGranularity deleteGranularity) { - createAndInitTable("id INT, dep STRING", "PARTITIONED BY (dep)", null /* empty */); + @TestTemplate + public void testMergeWithDVAndHistoricalPositionDeletes() { + assumeThat(formatVersion).isEqualTo(2); + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", DeleteGranularity.PARTITION); + createBranchIfNeeded(); + createOrReplaceView( + "source", IntStream.rangeClosed(1, 9).boxed().collect(Collectors.toList()), Encoders.INT()); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }\n" + + "{ \"id\": 3, \"dep\": \"hr\" }"); + append( + commitTarget(), + "{ \"id\": 4, \"dep\": \"hr\" }\n" + + "{ \"id\": 5, \"dep\": \"hr\" }\n" + + "{ \"id\": 6, \"dep\": \"hr\" }"); + + // Produce partition scoped deletes for the two modified files + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value and (id = 1 or id = 4) " + + "WHEN MATCHED THEN " + + " DELETE " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (-1, 'other')", + commitTarget()); + // Produce 1 file-scoped deletes for the second update + Map fileGranularityProps = + ImmutableMap.of(TableProperties.DELETE_GRANULARITY, DeleteGranularity.FILE.toString()); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(fileGranularityProps)); sql( - "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value and id = 5 " + + "WHEN MATCHED THEN " + + " UPDATE SET id = id + 2 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (-1, 'other')", + commitTarget()); + + Map updateFormatProperties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, "3"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(updateFormatProperties)); + + // Produce a DV which will contain 3 positions from the second data file + // 2 existing deleted positions from the earlier file-scoped and partition-scoped deletes + // and 1 new deleted position + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value and id = 6 " + + "WHEN MATCHED THEN " + + " UPDATE SET id = id + 1 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (-1, 'other')", + commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Set deleteFiles = + TestHelpers.deleteFiles(table, SnapshotUtil.latestSnapshot(table, branch)); + List dvs = + deleteFiles.stream().filter(ContentFileUtil::isDV).collect(Collectors.toList()); + assertThat(dvs).hasSize(1); + assertThat(dvs).allMatch(dv -> dv.recordCount() == 3); + } + + private void checkMergeDeleteGranularity(DeleteGranularity deleteGranularity) { + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", deleteGranularity); append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); append(tableName, "{ \"id\": 3, \"dep\": \"hr\" }\n" + "{ \"id\": 4, \"dep\": \"hr\" }"); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java index e9cc9d8541ad..477a2e73256b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java @@ -21,7 +21,11 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; +import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; @@ -30,6 +34,8 @@ import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.SnapshotUtil; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -40,24 +46,24 @@ public class TestMergeOnReadUpdate extends TestUpdate { @Override protected Map extraTableProperties() { return ImmutableMap.of( - TableProperties.FORMAT_VERSION, - "2", - TableProperties.UPDATE_MODE, - RowLevelOperationMode.MERGE_ON_READ.modeName()); + TableProperties.UPDATE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName()); } @TestTemplate public void testUpdateFileGranularity() { + assumeThat(formatVersion).isEqualTo(2); checkUpdateFileGranularity(DeleteGranularity.FILE); } @TestTemplate public void testUpdatePartitionGranularity() { + assumeThat(formatVersion).isEqualTo(2); checkUpdateFileGranularity(DeleteGranularity.PARTITION); } @TestTemplate - public void testUpdateFileGranularityMergesDeleteFiles() { + public void testPositionDeletesAreMaintainedDuringUpdate() { + assumeThat(formatVersion).isEqualTo(2); // Range distribution will produce partition scoped deletes which will not be cleaned up assumeThat(distributionMode).isNotEqualToIgnoringCase("range"); @@ -84,7 +90,8 @@ public void testUpdateFileGranularityMergesDeleteFiles() { } @TestTemplate - public void testUpdateUnpartitionedFileGranularityMergesDeleteFiles() { + public void testUnpartitionedPositionDeletesAreMaintainedDuringUpdate() { + assumeThat(formatVersion).isEqualTo(2); // Range distribution will produce partition scoped deletes which will not be cleaned up assumeThat(distributionMode).isNotEqualToIgnoringCase("range"); initTable("", DeleteGranularity.FILE); @@ -156,12 +163,56 @@ private void checkUpdateFileGranularity(DeleteGranularity deleteGranularity) { sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); } - private void initTable(String partitionedBy, DeleteGranularity deleteGranularity) { - createAndInitTable("id INT, dep STRING", partitionedBy, null /* empty */); + @TestTemplate + public void testUpdateWithDVAndHistoricalPositionDeletes() { + assumeThat(formatVersion).isEqualTo(2); + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", DeleteGranularity.PARTITION); + createBranchIfNeeded(); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }\n" + + "{ \"id\": 3, \"dep\": \"hr\" }"); + append( + commitTarget(), + "{ \"id\": 4, \"dep\": \"hr\" }\n" + + "{ \"id\": 5, \"dep\": \"hr\" }\n" + + "{ \"id\": 6, \"dep\": \"hr\" }"); + + // Produce partition scoped deletes for the two modified files + sql("UPDATE %s SET id = id - 1 WHERE id = 1 or id = 4", commitTarget()); + + // Produce 1 file-scoped deletes for the second update + Map fileGranularityProps = + ImmutableMap.of(TableProperties.DELETE_GRANULARITY, DeleteGranularity.FILE.toString()); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(fileGranularityProps)); + sql("UPDATE %s SET id = id + 2 WHERE id = 5", commitTarget()); + Map updateFormatProperties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, "3"); sql( - "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(updateFormatProperties)); + + // Produce a DV which will contain 3 positions from the second data file + // 2 existing deleted positions from the earlier file-scoped and partition-scoped deletes + // and 1 new deleted position + sql("UPDATE %s SET id = id + 1 where id = 6", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Set deleteFiles = + TestHelpers.deleteFiles(table, SnapshotUtil.latestSnapshot(table, branch)); + List dvs = + deleteFiles.stream().filter(ContentFileUtil::isDV).collect(Collectors.toList()); + assertThat(dvs).hasSize(1); + assertThat(dvs.get(0).recordCount()).isEqualTo(3); + } + + private void initTable(String partitionedBy, DeleteGranularity deleteGranularity) { + createTableWithDeleteGranularity("id INT, dep STRING", partitionedBy, deleteGranularity); append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); append(tableName, "{ \"id\": 3, \"dep\": \"hr\" }\n" + "{ \"id\": 4, \"dep\": \"hr\" }"); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java index 550bf41ce220..6332e303ad67 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.DataOperations.OVERWRITE; import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; import static org.apache.iceberg.SnapshotSummary.ADDED_FILES_PROP; import static org.apache.iceberg.SnapshotSummary.CHANGED_PARTITION_COUNT_PROP; import static org.apache.iceberg.SnapshotSummary.DELETED_FILES_PROP; @@ -170,6 +171,9 @@ public void testCoalesceUpdate() { // AQE detects that all shuffle blocks are small and processes them in 1 task // otherwise, there would be 200 tasks writing to the table validateProperty(snapshot, SnapshotSummary.ADDED_FILES_PROP, "1"); + } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { + validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); + validateProperty(snapshot, SnapshotSummary.ADDED_DVS_PROP, "4"); } else { // MoR UPDATE requests the deleted records to be range distributed by partition and `_file` // each task contains only 1 file and therefore writes only 1 shuffle block @@ -436,6 +440,8 @@ public void testUpdateWithoutCondition() { validateProperty(currentSnapshot, CHANGED_PARTITION_COUNT_PROP, "2"); validateProperty(currentSnapshot, DELETED_FILES_PROP, "3"); validateProperty(currentSnapshot, ADDED_FILES_PROP, ImmutableSet.of("2", "3")); + } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { + validateMergeOnRead(currentSnapshot, "2", "3", "2"); } else { validateMergeOnRead(currentSnapshot, "2", "2", "2"); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java index 2c8c26d80977..7ba8d558d5e7 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java @@ -39,9 +39,11 @@ import java.util.Map; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.exceptions.ValidationException; @@ -721,4 +723,9 @@ public DeleteGranularity deleteGranularity() { .parse(); return DeleteGranularity.fromString(valueAsString); } + + public boolean useDVs() { + TableOperations ops = ((HasTableOperations) table).operations(); + return ops.current().formatVersion() >= 3; + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index 18e483f23fc6..a361a7f1bae8 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -25,6 +25,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionScanTask; @@ -162,13 +163,13 @@ public void filter(Predicate[] predicates) { } } - protected Map rewritableDeletes() { + protected Map rewritableDeletes(boolean forDVs) { Map rewritableDeletes = Maps.newHashMap(); for (ScanTask task : tasks()) { FileScanTask fileScanTask = task.asFileScanTask(); for (DeleteFile deleteFile : fileScanTask.deletes()) { - if (ContentFileUtil.isFileScoped(deleteFile)) { + if (shouldRewrite(deleteFile, forDVs)) { rewritableDeletes .computeIfAbsent(fileScanTask.file().location(), ignored -> DeleteFileSet.create()) .add(deleteFile); @@ -179,6 +180,16 @@ protected Map rewritableDeletes() { return rewritableDeletes; } + // for DVs all position deletes must be rewritten + // for position deletes, only file-scoped deletes must be rewritten + private boolean shouldRewrite(DeleteFile deleteFile, boolean forDVs) { + if (forDVs) { + return deleteFile.content() != FileContent.EQUALITY_DELETES; + } + + return ContentFileUtil.isFileScoped(deleteFile); + } + // at this moment, Spark can only pass IN filters for a single attribute // if there are multiple filter attributes, Spark will pass two separate IN filters private Expression convertRuntimeFilters(Predicate[] predicates) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index 18020ee935b6..56ef3998e853 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -61,6 +61,7 @@ import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitioningDVWriter; import org.apache.iceberg.io.PartitioningWriter; import org.apache.iceberg.io.PositionDeltaWriter; import org.apache.iceberg.io.WriteResult; @@ -182,8 +183,8 @@ public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { } private Broadcast> broadcastRewritableDeletes() { - if (context.deleteGranularity() == DeleteGranularity.FILE && scan != null) { - Map rewritableDeletes = scan.rewritableDeletes(); + if (scan != null && shouldRewriteDeletes()) { + Map rewritableDeletes = scan.rewritableDeletes(context.useDVs()); if (rewritableDeletes != null && !rewritableDeletes.isEmpty()) { return sparkContext.broadcast(rewritableDeletes); } @@ -191,6 +192,11 @@ private Broadcast> broadcastRewritableDeletes() { return null; } + private boolean shouldRewriteDeletes() { + // deletes must be rewritten when there are DVs and file-scoped deletes + return context.useDVs() || context.deleteGranularity() == DeleteGranularity.FILE; + } + @Override public boolean useCommitCoordinator() { return false; @@ -474,7 +480,8 @@ protected PartitioningWriter newDataWriter( } } - // the spec requires position deletes to be ordered by file and position + // Use the DV writer for V3+ tables + // The spec requires position deletes to be ordered by file and position for V2 tables // use a fanout writer if the input is unordered no matter whether fanout writers are enabled // clustered writers assume that the position deletes are already ordered by file and position protected PartitioningWriter, DeleteWriteResult> newDeleteWriter( @@ -483,25 +490,21 @@ protected PartitioningWriter, DeleteWriteResult> new SparkFileWriterFactory writers, OutputFileFactory files, Context context) { - + Function previousDeleteLoader = + PreviousDeleteLoader.create(table, rewritableDeletes); FileIO io = table.io(); boolean inputOrdered = context.inputOrdered(); long targetFileSize = context.targetDeleteFileSize(); DeleteGranularity deleteGranularity = context.deleteGranularity(); - if (inputOrdered && rewritableDeletes == null) { + if (context.useDVs()) { + return new PartitioningDVWriter<>(files, previousDeleteLoader); + } else if (inputOrdered && rewritableDeletes == null) { return new ClusteredPositionDeleteWriter<>( writers, files, io, targetFileSize, deleteGranularity); } else { return new FanoutPositionOnlyDeleteWriter<>( - writers, - files, - io, - targetFileSize, - deleteGranularity, - rewritableDeletes != null - ? new PreviousDeleteLoader(table, rewritableDeletes) - : path -> null /* no previous file scoped deletes */); + writers, files, io, targetFileSize, deleteGranularity, previousDeleteLoader); } } } @@ -510,7 +513,7 @@ private static class PreviousDeleteLoader implements Function deleteFiles; private final DeleteLoader deleteLoader; - PreviousDeleteLoader(Table table, Map deleteFiles) { + private PreviousDeleteLoader(Table table, Map deleteFiles) { this.deleteFiles = deleteFiles; this.deleteLoader = new BaseDeleteLoader( @@ -528,6 +531,15 @@ public PositionDeleteIndex apply(CharSequence path) { return deleteLoader.loadPositionDeletes(deleteFileSet, path); } + + public static Function create( + Table table, Map deleteFiles) { + if (deleteFiles == null) { + return path -> null; + } + + return new PreviousDeleteLoader(table, deleteFiles); + } } private static class DeleteOnlyDeltaWriter extends BaseDeltaWriter { @@ -774,6 +786,7 @@ private static class Context implements Serializable { private final String queryId; private final boolean useFanoutWriter; private final boolean inputOrdered; + private final boolean useDVs; Context( Schema dataSchema, @@ -792,6 +805,7 @@ private static class Context implements Serializable { this.queryId = info.queryId(); this.useFanoutWriter = writeConf.useFanoutWriter(writeRequirements); this.inputOrdered = writeRequirements.hasOrdering(); + this.useDVs = writeConf.useDVs(); } Schema dataSchema() { @@ -838,6 +852,10 @@ boolean inputOrdered() { return inputOrdered; } + boolean useDVs() { + return useDVs; + } + int specIdOrdinal() { return metadataSparkType.fieldIndex(MetadataColumns.SPEC_ID.name()); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index d64ca588f202..f9f24834546f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -50,6 +50,7 @@ import org.apache.iceberg.FileScanTask; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; import org.apache.iceberg.io.CloseableIterable; @@ -811,6 +812,16 @@ public static Set deleteFiles(Table table) { return deleteFiles; } + public static Set deleteFiles(Table table, Snapshot snapshot) { + DeleteFileSet deleteFiles = DeleteFileSet.create(); + + for (FileScanTask task : table.newScan().useSnapshot(snapshot.snapshotId()).planFiles()) { + deleteFiles.addAll(task.deletes()); + } + + return deleteFiles; + } + public static Set reachableManifestPaths(Table table) { return StreamSupport.stream(table.snapshots().spliterator(), false) .flatMap(s -> s.allManifests(table.io()).stream()) From f931a3dcc5092533a1e383b926f7123dd5307d7b Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Fri, 6 Dec 2024 12:09:29 -0800 Subject: [PATCH 227/313] Infra: Add 1.7.1 to issue template (#11711) --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index 79b492fff7aa..61b21cb49d38 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -28,7 +28,8 @@ body: description: What Apache Iceberg version are you using? multiple: false options: - - "1.7.0 (latest release)" + - "1.7.1 (latest release)" + - "1.7.0" - "1.6.1" - "1.6.0" - "1.5.2" From 2210e2887348453e8d787c7fcc9ffc8d6f276739 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Fri, 6 Dec 2024 13:49:27 -0800 Subject: [PATCH 228/313] Update ASF doap.rdf to Release 1.7.1 (#11712) --- doap.rdf | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doap.rdf b/doap.rdf index 4b7418534644..a7409329ca1c 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ - 1.7.0 - 2024-11-08 - 1.7.0 + 1.7.1 + 2024-12-06 + 1.7.1 From deeb04b2c1ceb9bb8ebee79254f2c98ab5b95b15 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 6 Dec 2024 18:15:02 -0800 Subject: [PATCH 229/313] Spark 3.3,3.4: Align RewritePositionDeleteFilesSparkAction filter with Spark case sensitivity (#11710) --- .../RewritePositionDeleteFilesSparkAction.java | 5 ++++- .../TestRewritePositionDeleteFilesAction.java | 18 +++++++++++++++++- .../RewritePositionDeleteFilesSparkAction.java | 5 ++++- .../TestRewritePositionDeleteFilesAction.java | 18 +++++++++++++++++- 4 files changed, 42 insertions(+), 4 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java index 8bc67c35c3d6..12113bd2c99a 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java @@ -59,6 +59,7 @@ import org.apache.iceberg.relocated.com.google.common.math.IntMath; import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.PartitionUtil; import org.apache.iceberg.util.PropertyUtil; @@ -92,11 +93,13 @@ public class RewritePositionDeleteFilesSparkAction private int maxCommits; private boolean partialProgressEnabled; private RewriteJobOrder rewriteJobOrder; + private boolean caseSensitive; RewritePositionDeleteFilesSparkAction(SparkSession spark, Table table) { super(spark); this.table = table; this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table); + this.caseSensitive = SparkUtil.caseSensitive(spark); } @Override @@ -158,7 +161,7 @@ private StructLikeMap>> planFileGroups() { private CloseableIterable planFiles(Table deletesTable) { PositionDeletesBatchScan scan = (PositionDeletesBatchScan) deletesTable.newBatchScan(); return CloseableIterable.transform( - scan.baseTableFilter(filter).ignoreResiduals().planFiles(), + scan.baseTableFilter(filter).caseSensitive(caseSensitive).ignoreResiduals().planFiles(), task -> (PositionDeletesScanTask) task); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index aa2817e8753d..43b762813de2 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.spark.sql.functions.expr; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.Arrays; @@ -71,6 +72,7 @@ import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; import org.junit.After; import org.junit.Assert; @@ -228,7 +230,8 @@ public void testRewriteFilter() throws Exception { Expression filter = Expressions.and( Expressions.greaterThan("c3", "0"), // should have no effect - Expressions.or(Expressions.equal("c1", 1), Expressions.equal("c1", 2))); + // "C1" should work because Spark defaults case sensitivity to false. + Expressions.or(Expressions.equal("C1", 1), Expressions.equal("C1", 2))); Result result = SparkActions.get(spark) @@ -250,6 +253,19 @@ public void testRewriteFilter() throws Exception { List actualDeletes = deleteRecords(table); assertEquals("Rows must match", expectedRecords, actualRecords); assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + + withSQLConf( + ImmutableMap.of(SQLConf.CASE_SENSITIVE().key(), "true"), + () -> { + assertThatThrownBy( + () -> + SparkActions.get(spark) + .rewritePositionDeletes(table) + .filter(filter) + .execute()) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot find field 'C1' in struct"); + }); } @Test diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java index ccf874716db0..6a63d2f63e4b 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java @@ -59,6 +59,7 @@ import org.apache.iceberg.relocated.com.google.common.math.IntMath; import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.PartitionUtil; import org.apache.iceberg.util.PropertyUtil; @@ -92,11 +93,13 @@ public class RewritePositionDeleteFilesSparkAction private int maxCommits; private boolean partialProgressEnabled; private RewriteJobOrder rewriteJobOrder; + private boolean caseSensitive; RewritePositionDeleteFilesSparkAction(SparkSession spark, Table table) { super(spark); this.table = table; this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table); + this.caseSensitive = SparkUtil.caseSensitive(spark); } @Override @@ -159,7 +162,7 @@ private CloseableIterable planFiles(Table deletesTable) PositionDeletesBatchScan scan = (PositionDeletesBatchScan) deletesTable.newBatchScan(); return CloseableIterable.transform( - scan.baseTableFilter(filter).ignoreResiduals().planFiles(), + scan.baseTableFilter(filter).caseSensitive(caseSensitive).ignoreResiduals().planFiles(), task -> (PositionDeletesScanTask) task); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 7be300e84fc6..5820d46703e2 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.spark.sql.functions.expr; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.Arrays; @@ -72,6 +73,7 @@ import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; import org.junit.After; import org.junit.Assert; @@ -265,7 +267,8 @@ public void testRewriteFilter() throws Exception { Expression filter = Expressions.and( Expressions.greaterThan("c3", "0"), // should have no effect - Expressions.or(Expressions.equal("c1", 1), Expressions.equal("c1", 2))); + // "C1" should work because Spark defaults case sensitivity to false. + Expressions.or(Expressions.equal("C1", 1), Expressions.equal("C1", 2))); Result result = SparkActions.get(spark) @@ -287,6 +290,19 @@ public void testRewriteFilter() throws Exception { List actualDeletes = deleteRecords(table); assertEquals("Rows must match", expectedRecords, actualRecords); assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + + withSQLConf( + ImmutableMap.of(SQLConf.CASE_SENSITIVE().key(), "true"), + () -> { + assertThatThrownBy( + () -> + SparkActions.get(spark) + .rewritePositionDeletes(table) + .filter(filter) + .execute()) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot find field 'C1' in struct"); + }); } @Test From 5d1dc1aa52dd92180bf0f3bc0b9eb57dbd0690e3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Dec 2024 11:35:58 +0100 Subject: [PATCH 230/313] Build: Bump software.amazon.awssdk:bom from 2.29.23 to 2.29.29 (#11723) Bumps software.amazon.awssdk:bom from 2.29.23 to 2.29.29. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 1bf625cbf8bb..edfec7aff976 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.29.23" +awssdk-bom = "2.29.29" azuresdk-bom = "1.2.29" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" From b39253fcd4fdde765ba3a83f84b4160b9bdcb2a8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Dec 2024 11:48:51 +0100 Subject: [PATCH 231/313] Build: Bump com.google.cloud:libraries-bom from 26.50.0 to 26.51.0 (#11724) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.50.0 to 26.51.0. - [Release notes](https://github.com/googleapis/java-cloud-bom/releases) - [Changelog](https://github.com/googleapis/java-cloud-bom/blob/main/release-please-config.json) - [Commits](https://github.com/googleapis/java-cloud-bom/compare/v26.50.0...v26.51.0) --- updated-dependencies: - dependency-name: com.google.cloud:libraries-bom dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index edfec7aff976..943f024b24cc 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -44,7 +44,7 @@ findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} flink120 = { strictly = "1.20.0"} -google-libraries-bom = "26.50.0" +google-libraries-bom = "26.51.0" guava = "33.3.1-jre" hadoop2 = "2.7.3" hadoop3 = "3.4.1" From 3eed132fd5be13237e245d0a528e64dcfa9f68d4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Dec 2024 12:07:45 +0100 Subject: [PATCH 232/313] Build: Bump mkdocs-material from 9.5.46 to 9.5.47 (#11726) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.46 to 9.5.47. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.5.46...9.5.47) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 59dcb22c7e3e..63985d948b6a 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.3.7 -mkdocs-material==9.5.46 +mkdocs-material==9.5.47 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.2 From 0699c8db8579d7f12c188f9af2c2ab6e923fa84d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 9 Dec 2024 14:07:38 +0100 Subject: [PATCH 233/313] Add C++ to the list of languages in `doap.rdf` (#11714) From the list of accepted values: https://projects.apache.org/validation.json --- doap.rdf | 1 + 1 file changed, 1 insertion(+) diff --git a/doap.rdf b/doap.rdf index a7409329ca1c..8a9254ce2e98 100644 --- a/doap.rdf +++ b/doap.rdf @@ -36,6 +36,7 @@ Python Go Rust + C++ From 0662373a676c40dabdfe66b8672b404cd53bb878 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Dec 2024 14:11:59 +0100 Subject: [PATCH 234/313] Build: Bump com.azure:azure-sdk-bom from 1.2.29 to 1.2.30 (#11725) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.2.29 to 1.2.30. - [Release notes](https://github.com/azure/azure-sdk-for-java/releases) - [Commits](https://github.com/azure/azure-sdk-for-java/compare/azure-sdk-bom_1.2.29...azure-sdk-bom_1.2.30) --- updated-dependencies: - dependency-name: com.azure:azure-sdk-bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 943f024b24cc..1a290b1b40a6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -30,7 +30,7 @@ avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" awssdk-bom = "2.29.29" -azuresdk-bom = "1.2.29" +azuresdk-bom = "1.2.30" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" calcite = "1.10.0" From b18ab74c3c2ecddc8f17c9bdff625996a8ee600c Mon Sep 17 00:00:00 2001 From: dominikhei <105610163+dominikhei@users.noreply.github.com> Date: Mon, 9 Dec 2024 14:24:36 +0100 Subject: [PATCH 235/313] Add `curl` to the `iceberg-rest-fixture` Docker image (#11705) * Added command to install curl * Added command to cleanup apt cache * Added health probing * Update healthcheck --------- Co-authored-by: Fokko Driesprong --- docker/iceberg-rest-fixture/Dockerfile | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docker/iceberg-rest-fixture/Dockerfile b/docker/iceberg-rest-fixture/Dockerfile index 30189b97c53e..39d959cd866e 100644 --- a/docker/iceberg-rest-fixture/Dockerfile +++ b/docker/iceberg-rest-fixture/Dockerfile @@ -22,7 +22,10 @@ FROM azul/zulu-openjdk:17-jre-headless # Set up the user and group RUN set -xeu && \ groupadd iceberg --gid 1000 && \ - useradd iceberg --uid 1000 --gid 1000 --create-home + useradd iceberg --uid 1000 --gid 1000 --create-home && \ + apt-get update && \ + apt-get install -y --no-install-recommends curl && \ + rm -rf /var/lib/apt/lists/* # Working directory for the application WORKDIR /usr/lib/iceberg-rest @@ -36,6 +39,10 @@ ENV CATALOG_JDBC_USER=user ENV CATALOG_JDBC_PASSWORD=password ENV REST_PORT=8181 +# Healthcheck for the iceberg rest service +HEALTHCHECK --retries=10 --interval=1s \ + CMD curl --fail http://localhost:$REST_PORT/v1/config || exit 1 + EXPOSE $REST_PORT USER iceberg:iceberg ENV LANG=en_US.UTF-8 From 70d87f1750627b14b3b25a0216a97db86a786992 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Dec 2024 14:45:16 +0100 Subject: [PATCH 236/313] Build: Bump nessie from 0.100.2 to 0.101.0 (#11722) Bumps `nessie` from 0.100.2 to 0.101.0. Updates `org.projectnessie.nessie:nessie-client` from 0.100.2 to 0.101.0 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.100.2 to 0.101.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.100.2 to 0.101.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.100.2 to 0.101.0 --- updated-dependencies: - dependency-name: org.projectnessie.nessie:nessie-client dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-jaxrs-testextension dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-testextension dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 1a290b1b40a6..3981ac05ff11 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -70,7 +70,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.100.2" +nessie = "0.101.0" netty-buffer = "4.1.115.Final" netty-buffer-compat = "4.1.115.Final" object-client-bundle = "3.3.2" From 2b2efd78f622df971e96ec1b72fe64fca9b4f7c2 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Mon, 9 Dec 2024 10:35:56 -0800 Subject: [PATCH 237/313] docs: 1.7.1 Release notes (#11717) * Docs: Add release notes for 1.7.1 * Move past releases marker --- site/docs/releases.md | 24 ++++++++++++++++++++++-- site/mkdocs.yml | 2 +- site/nav.yml | 1 + 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 5346d2a476d8..27ad3b8bfc4c 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -72,8 +72,30 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: ``` +### 1.7.1 release + +Apache Iceberg 1.7.1 was released on December 6, 2024. + +The 1.7.1 release contains bug fixes and new features. For full release notes visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.7.1) + +* Core + - Revert "Use encoding/decoding methods for namespaces and deprecate Splitter/Joiner" ([\#11574](https://github.com/apache/iceberg/pull/11574)) + - Revert "Update TableMetadataParser to ensure all streams closed" ([\#11621](https://github.com/apache/iceberg/pull/11621)) +* Azure + - Fix ADLSLocation file parsing ([\#11395](https://github.com/apache/iceberg/pull/11395)) + - Support WASB scheme in ADLSFileIO ([\#11504](https://github.com/apache/iceberg/pull/11504)) +* Spark + - Fix NotSerializableException when migrating Spark tables ([\#11157](https://github.com/apache/iceberg/pull/11157)) + - Fix changelog table bug for start time older than current snapshot ([\#11564](https://github.com/apache/iceberg/pull/11564)) +* Kafka Connect + - Fix Hadoop dependency exclusion ([\#11516](https://github.com/apache/iceberg/pull/11516)) + +## Past releases + ### 1.7.0 release +Apache Iceberg 1.7.0 was released on November 8, 2024. + The 1.7.0 release contains fixes, dependency updates, and new features. For full release notes please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.7.0). An abridged list follows * Deprecation / End of Support @@ -255,8 +277,6 @@ The 1.6.0 release contains fixes, dependency updates, and new features (like Kaf For more details, please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.6.0). -## Past releases - ### 1.5.2 release Apache Iceberg 1.5.2 was released on May 9, 2024. diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 646c87392826..3eed2bd360fa 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -78,7 +78,7 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.7.0' + icebergVersion: '1.7.1' nessieVersion: '0.92.1' flinkVersion: '1.19.0' flinkVersionMajor: '1.19' diff --git a/site/nav.yml b/site/nav.yml index e54bb6260a77..da6a56bc6de2 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -23,6 +23,7 @@ nav: - Docs: - nightly: '!include docs/docs/nightly/mkdocs.yml' - latest: '!include docs/docs/latest/mkdocs.yml' + - 1.7.1: '!include docs/docs/1.7.1/mkdocs.yml' - 1.7.0: '!include docs/docs/1.7.0/mkdocs.yml' - 1.6.1: '!include docs/docs/1.6.1/mkdocs.yml' - 1.6.0: '!include docs/docs/1.6.0/mkdocs.yml' From d402f83fc7b224b21242c506cf503e5bcbc8c867 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 9 Dec 2024 11:20:45 -0800 Subject: [PATCH 238/313] Docs: Add guidelines for contributors to become committers (#11670) Co-authored-by: Fokko Driesprong Co-authored-by: Eduard Tudenhoefner --- site/docs/community.md | 57 +++++++++++++++++++++++++++++++++++++++++- 1 file changed, 56 insertions(+), 1 deletion(-) diff --git a/site/docs/community.md b/site/docs/community.md index ac028a46e9c3..421d41f4d969 100644 --- a/site/docs/community.md +++ b/site/docs/community.md @@ -76,7 +76,9 @@ Iceberg has four mailing lists: The Apache Iceberg community is built on the principles described in the [Apache Way](https://www.apache.org/theapacheway/index.html) and all who engage with the community are expected to be respectful, open, come with the best interests of the community in mind, -and abide by the Apache Foundation [Code of Conduct](https://www.apache.org/foundation/policies/conduct.html). +and abide by the Apache Software Foundation [Code of Conduct](https://www.apache.org/foundation/policies/conduct.html). + +More information specific to the Apache Iceberg community is in the next section, [the Path from Contributor to Committer](#the-path-from-contributor-to-committer). ### Participants with Corporate Interests @@ -100,3 +102,56 @@ Recruitment of community members should not be conducted through direct messages related to contributing to or using Iceberg can be posted to the `#jobs` channel. For questions regarding any of the guidelines above, please contact a PMC member + +## The Path from Contributor to Committer + +Many contributors have questions about how to become a committer. This section outlines what committers do and how they are invited. + +### What are the responsibilities of a committer? + +In the Iceberg project, committers are community members that can review and commit changes to Iceberg repositories. Reviewing is the primary responsibility of committers. + +### How are new committers added? + +Starting from the foundation guidelines, committers are nominated and discussed by the PMC, which uses a consensus vote to confirm a new committer. This vote is the only formal requirement in the Iceberg community — there are no other requirements, such as a minimum period of time or a minimum number of contributions. Similarly, there is no length of time or number of commits that automatically qualify someone to be a committer. + +Committers are added when someone has built trust with PMC members that they have good judgment and are a reliable reviewer. + +### What does the PMC look for? + +PMC members typically look for candidates to have demonstrated a few qualities: + +* **Conduct** — Committers are representatives of the project and are expected to follow the [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct.html). +* **Judgment** — Committers should know the areas where they are qualified to evaluate a change and when to bring in other opinions. +* **Quality** — Personal contributions are a strong signal. Contributions that don’t require major help demonstrate the context and understanding needed to reliably review changes from others. If a contributor often needs guidance, they are probably not ready to guide others. +* **Consistency** — Reviewing is the primary responsibility of a committer. A committer should demonstrate they will consistently apply their context and understanding to help contributors get changes in and ensure those changes are high quality. + +### How do I demonstrate those qualities? + +To be a committer, a candidate should act like a committer so that PMC members can evaluate the qualities above. PMC members will ask questions like these: + +* Has the candidate been a good representative of the project in mailing lists, Slack, github, and other discussion forums? +* Has the candidate followed the ASF Code of Conduct when working with others? +* Has the candidate made independent material contributions to the community that show expertise? +* Have the candidate’s contributions been stable and maintainable? +* Has the candidate’s work required extensive review or significant refactoring due to misunderstandings of the project’s objectives? +* Does the candidate apply the standards and conventions of the project by following existing patterns and using already included libraries? +* Has the candidate participated in design discussions for new features? +* Has the candidate asked for help when reviewing changes outside their area of expertise? +* How diverse are the contributors that the candidate reviewed? +* Does the candidate raise potentially problematic changes to the dev list? + +### How can I be a committer? + +You can always reach out to PMC members for feedback and guidance if you have questions. + +There is no single path to becoming a committer. For example, people contributing to Python are often implicitly trusted not to start reviewing changes to other languages. Similarly, some areas of a project require more context than others. + +Keep in mind that it’s best not to compare your contributions to others. Instead, focus on demonstrating quality and judgment. + +### How many contributions does it take to become a committer? + +The number of contributions is not what matters — the quality of those contributions (including reviews!) is what demonstrates that a contributor is ready to be a committer. + +You can always reach out to PMC members directly or using private@iceberg.apache.org for feedback and guidance if you have questions. + From 28e81809e64b3d6c5b1f8d40f53fcb5edc1ec3e9 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 10 Dec 2024 20:36:12 +0100 Subject: [PATCH 239/313] Core, Flink, Spark: Drop deprecated APIs scheduled for removal in 1.8.0 (#11721) --- .palantir/revapi.yml | 26 +++ .../parquet/VectorizedColumnIterator.java | 26 --- ...dDictionaryEncodedParquetValuesReader.java | 21 -- .../parquet/VectorizedPageIterator.java | 42 ---- ...ectorizedParquetDefinitionLevelReader.java | 51 ----- .../org/apache/iceberg/deletes/Deletes.java | 78 +------- .../iceberg/deletes/TestPositionFilter.java | 32 +--- .../iceberg/flink/data/FlinkAvroReader.java | 181 ------------------ .../TestFlinkAvroDeprecatedReaderWriter.java | 38 ---- .../iceberg/flink/data/TestRowProjection.java | 69 ++----- .../iceberg/flink/data/FlinkAvroReader.java | 181 ------------------ .../TestFlinkAvroDeprecatedReaderWriter.java | 38 ---- .../iceberg/flink/data/TestRowProjection.java | 67 ++----- .../iceberg/flink/data/FlinkAvroReader.java | 181 ------------------ .../TestFlinkAvroDeprecatedReaderWriter.java | 38 ---- .../iceberg/flink/data/TestRowProjection.java | 68 ++----- .../iceberg/spark/data/SparkAvroReader.java | 180 ----------------- 17 files changed, 85 insertions(+), 1232 deletions(-) delete mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java delete mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java delete mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java delete mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index fade79326a49..18c63fbe7bb1 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1145,6 +1145,32 @@ acceptedBreaks: new: "method org.apache.iceberg.BaseMetastoreOperations.CommitStatus org.apache.iceberg.BaseMetastoreTableOperations::checkCommitStatus(java.lang.String,\ \ org.apache.iceberg.TableMetadata)" justification: "Removing deprecated code" + "1.7.0": + org.apache.iceberg:iceberg-core: + - code: "java.method.removed" + old: "method org.apache.iceberg.deletes.PositionDeleteIndex\ + \ org.apache.iceberg.deletes.Deletes::toPositionIndex(java.lang.CharSequence,\ + \ java.util.List>)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.deletes.PositionDeleteIndex\ + \ org.apache.iceberg.deletes.Deletes::toPositionIndex(java.lang.CharSequence,\ + \ java.util.List>, java.util.concurrent.ExecutorService)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.CloseableIterable org.apache.iceberg.deletes.Deletes::streamingFilter(org.apache.iceberg.io.CloseableIterable,\ + \ java.util.function.Function, org.apache.iceberg.io.CloseableIterable)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.CloseableIterable org.apache.iceberg.deletes.Deletes::streamingFilter(org.apache.iceberg.io.CloseableIterable,\ + \ java.util.function.Function, org.apache.iceberg.io.CloseableIterable,\ + \ org.apache.iceberg.deletes.DeleteCounter)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.CloseableIterable org.apache.iceberg.deletes.Deletes::streamingMarker(org.apache.iceberg.io.CloseableIterable,\ + \ java.util.function.Function, org.apache.iceberg.io.CloseableIterable,\ + \ java.util.function.Consumer)" + justification: "Removing deprecated code" apache-iceberg-0.14.0: org.apache.iceberg:iceberg-api: - code: "java.class.defaultSerializationChanged" diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java index 54445e424508..8e52fd1a28c3 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java @@ -214,24 +214,6 @@ protected int nextBatchOf( } } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - public class FixedWidthTypeBinaryBatchReader extends BatchReader { - @Override - protected int nextBatchOf( - final FieldVector vector, - final int expectedBatchSize, - final int numValsInVector, - final int typeWidth, - NullabilityHolder holder) { - return vectorizedPageIterator - .fixedWidthBinaryPageReader() - .nextBatch(vector, expectedBatchSize, numValsInVector, typeWidth, holder); - } - } - public class BooleanBatchReader extends BatchReader { @Override protected int nextBatchOf( @@ -282,14 +264,6 @@ public VarWidthTypeBatchReader varWidthTypeBatchReader() { return new VarWidthTypeBatchReader(); } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - public FixedWidthTypeBinaryBatchReader fixedWidthTypeBinaryBatchReader() { - return new FixedWidthTypeBinaryBatchReader(); - } - public BooleanBatchReader booleanBatchReader() { return new BooleanBatchReader(); } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java index 848bb2d936c3..1a00137fad70 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java @@ -137,19 +137,6 @@ protected void nextVal( } } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - class FixedWidthBinaryDictEncodedReader extends BaseDictEncodedReader { - @Override - protected void nextVal( - FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - ByteBuffer buffer = dict.decodeToBinary(currentVal).toByteBuffer(); - vector.getDataBuffer().setBytes((long) idx * typeWidth, buffer); - } - } - class VarWidthBinaryDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( @@ -203,14 +190,6 @@ public DoubleDictEncodedReader doubleDictEncodedReader() { return new DoubleDictEncodedReader(); } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - public FixedWidthBinaryDictEncodedReader fixedWidthBinaryDictEncodedReader() { - return new FixedWidthBinaryDictEncodedReader(); - } - public VarWidthBinaryDictEncodedReader varWidthBinaryDictEncodedReader() { return new VarWidthBinaryDictEncodedReader(); } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java index d36521273d63..b97eb1545550 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java @@ -21,7 +21,6 @@ import java.io.IOException; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.IntVector; -import org.apache.arrow.vector.VarBinaryVector; import org.apache.iceberg.arrow.vectorized.NullabilityHolder; import org.apache.iceberg.parquet.BasePageIterator; import org.apache.iceberg.parquet.ParquetUtil; @@ -417,39 +416,6 @@ protected void nextDictEncodedVal( } } - /** - * Method for reading batches of fixed width binary type (e.g. BYTE[7]). Spark does not support - * fixed width binary data type. To work around this limitation, the data is read as fixed width - * binary from parquet and stored in a {@link VarBinaryVector} in Arrow. - * - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - class FixedWidthBinaryPageReader extends BasePageReader { - @Override - protected void nextVal( - FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { - vectorizedDefinitionLevelReader - .fixedWidthBinaryReader() - .nextBatch(vector, numVals, typeWidth, batchSize, holder, plainValuesReader); - } - - @Override - protected void nextDictEncodedVal( - FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { - vectorizedDefinitionLevelReader - .fixedWidthBinaryReader() - .nextDictEncodedBatch( - vector, - numVals, - typeWidth, - batchSize, - holder, - dictionaryEncodedValuesReader, - dictionary); - } - } - /** Method for reading batches of booleans. */ class BooleanPageReader extends BasePageReader { @Override @@ -499,14 +465,6 @@ VarWidthTypePageReader varWidthTypePageReader() { return new VarWidthTypePageReader(); } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - FixedWidthBinaryPageReader fixedWidthBinaryPageReader() { - return new FixedWidthBinaryPageReader(); - } - BooleanPageReader booleanPageReader() { return new BooleanPageReader(); } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java index e8ec7bee8f62..4df648ee3dfd 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java @@ -27,7 +27,6 @@ import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.IntVector; -import org.apache.arrow.vector.VarBinaryVector; import org.apache.iceberg.arrow.vectorized.NullabilityHolder; import org.apache.iceberg.parquet.ParquetUtil; import org.apache.iceberg.parquet.ValuesAsBytesReader; @@ -493,48 +492,6 @@ protected void nextDictEncodedVal( } } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - class FixedWidthBinaryReader extends BaseReader { - @Override - protected void nextVal( - FieldVector vector, - int idx, - ValuesAsBytesReader valuesReader, - int typeWidth, - byte[] byteArray) { - ByteBuffer buffer = valuesReader.getBuffer(typeWidth); - ((VarBinaryVector) vector) - .setSafe( - idx, - buffer.array(), - buffer.position() + buffer.arrayOffset(), - buffer.limit() - buffer.position()); - } - - @Override - protected void nextDictEncodedVal( - FieldVector vector, - int idx, - VectorizedDictionaryEncodedParquetValuesReader reader, - int numValuesToRead, - Dictionary dict, - NullabilityHolder nullabilityHolder, - int typeWidth, - Mode mode) { - if (Mode.RLE.equals(mode)) { - reader - .fixedWidthBinaryDictEncodedReader() - .nextBatch(vector, idx, numValuesToRead, dict, nullabilityHolder, typeWidth); - } else if (Mode.PACKED.equals(mode)) { - ByteBuffer buffer = dict.decodeToBinary(reader.readInteger()).toByteBuffer(); - vector.getDataBuffer().setBytes((long) idx * typeWidth, buffer); - } - } - } - class FixedSizeBinaryReader extends BaseReader { @Override protected void nextVal( @@ -736,14 +693,6 @@ TimestampInt96Reader timestampInt96Reader() { return new TimestampInt96Reader(); } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - FixedWidthBinaryReader fixedWidthBinaryReader() { - return new FixedWidthBinaryReader(); - } - FixedSizeBinaryReader fixedSizeBinaryReader() { return new FixedSizeBinaryReader(); } diff --git a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java index 0838e9c0dd6e..46df91982ab7 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java +++ b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.List; -import java.util.concurrent.ExecutorService; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Predicate; @@ -38,10 +37,8 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceMap; import org.apache.iceberg.util.Filter; -import org.apache.iceberg.util.ParallelIterable; import org.apache.iceberg.util.SortedMerge; import org.apache.iceberg.util.StructLikeSet; -import org.apache.iceberg.util.ThreadPools; public class Deletes { @@ -132,9 +129,8 @@ public static CharSequenceMap toPosi /** * Builds a map of position delete indexes by path. * - *

    Unlike {@link #toPositionIndex(CharSequence, List)}, this method builds a position delete - * index for each referenced data file and does not filter deletes. This can be useful when the - * entire delete file content is needed (e.g. caching). + *

    This method builds a position delete index for each referenced data file and does not filter + * deletes. This can be useful when the entire delete file content is needed (e.g. caching). * * @param posDeletes position deletes * @param file the source delete file for the deletes @@ -173,37 +169,6 @@ private static CloseableIterable extractPositions( return CloseableIterable.transform(filteredRows, row -> (Long) POSITION_ACCESSOR.get(row)); } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0; use delete loaders. - */ - @Deprecated - public static PositionDeleteIndex toPositionIndex( - CharSequence dataLocation, List> deleteFiles) { - return toPositionIndex(dataLocation, deleteFiles, ThreadPools.getDeleteWorkerPool()); - } - - /** - * @deprecated since 1.7.0, will be removed in 1.8.0; use delete loaders. - */ - @Deprecated - public static PositionDeleteIndex toPositionIndex( - CharSequence dataLocation, - List> deleteFiles, - ExecutorService deleteWorkerPool) { - DataFileFilter locationFilter = new DataFileFilter<>(dataLocation); - List> positions = - Lists.transform( - deleteFiles, - deletes -> - CloseableIterable.transform( - locationFilter.filter(deletes), row -> (Long) POSITION_ACCESSOR.get(row))); - if (positions.size() > 1 && deleteWorkerPool != null) { - return toPositionIndex(new ParallelIterable<>(positions, deleteWorkerPool)); - } else { - return toPositionIndex(CloseableIterable.concat(positions)); - } - } - public static PositionDeleteIndex toPositionIndex(CloseableIterable posDeletes) { return toPositionIndex(posDeletes, ImmutableList.of()); } @@ -219,45 +184,6 @@ private static PositionDeleteIndex toPositionIndex( } } - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - public static CloseableIterable streamingFilter( - CloseableIterable rows, - Function rowToPosition, - CloseableIterable posDeletes) { - return streamingFilter(rows, rowToPosition, posDeletes, new DeleteCounter()); - } - - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - public static CloseableIterable streamingFilter( - CloseableIterable rows, - Function rowToPosition, - CloseableIterable posDeletes, - DeleteCounter counter) { - PositionDeleteIndex positionIndex = toPositionIndex(posDeletes); - Predicate isDeleted = row -> positionIndex.isDeleted(rowToPosition.apply(row)); - return filterDeleted(rows, isDeleted, counter); - } - - /** - * @deprecated since 1.7.0, will be removed in 1.8.0. - */ - @Deprecated - public static CloseableIterable streamingMarker( - CloseableIterable rows, - Function rowToPosition, - CloseableIterable posDeletes, - Consumer markRowDeleted) { - PositionDeleteIndex positionIndex = toPositionIndex(posDeletes); - Predicate isDeleted = row -> positionIndex.isDeleted(rowToPosition.apply(row)); - return markDeleted(rows, isDeleted, markRowDeleted); - } - public static CloseableIterable deletePositions( CharSequence dataLocation, CloseableIterable deleteFile) { return deletePositions(dataLocation, ImmutableList.of(deleteFile)); diff --git a/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java b/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java index 2a629b2dc2b3..5def0c70bb04 100644 --- a/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java +++ b/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java @@ -21,11 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadPoolExecutor; import java.util.function.Predicate; -import java.util.stream.Stream; import org.apache.avro.util.Utf8; import org.apache.iceberg.StructLike; import org.apache.iceberg.TestHelpers.Row; @@ -33,10 +29,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; public class TestPositionFilter { @Test @@ -134,16 +127,8 @@ public void testPositionSetRowFilter() { .containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L)); } - static Stream executorServiceProvider() { - return Stream.of( - null, - MoreExecutors.getExitingExecutorService( - (ThreadPoolExecutor) Executors.newFixedThreadPool(4))); - } - - @ParameterizedTest - @MethodSource("executorServiceProvider") - public void testCombinedPositionSetRowFilter(ExecutorService executorService) { + @Test + public void testCombinedPositionSetRowFilter() { CloseableIterable positionDeletes1 = CloseableIterable.withNoopClose( Lists.newArrayList( @@ -175,13 +160,14 @@ public void testCombinedPositionSetRowFilter(ExecutorService executorService) { Row.of(8L, "i"), Row.of(9L, "j"))); + CloseableIterable positions = + CloseableIterable.transform( + CloseableIterable.filter( + CloseableIterable.concat(ImmutableList.of(positionDeletes1, positionDeletes2)), + row -> "file_a.avro".equals(row.get(0, String.class))), + row -> row.get(1, Long.class)); Predicate isDeleted = - row -> - Deletes.toPositionIndex( - "file_a.avro", - ImmutableList.of(positionDeletes1, positionDeletes2), - executorService) - .isDeleted(row.get(0, Long.class)); + row -> Deletes.toPositionIndex(positions).isDeleted(row.get(0, Long.class)); CloseableIterable actual = CloseableIterable.filter(rows, isDeleted.negate()); assertThat(Iterables.transform(actual, row -> row.get(0, Long.class))) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java deleted file mode 100644 index baae91dd1882..000000000000 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.data; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.Decoder; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.avro.AvroSchemaWithTypeVisitor; -import org.apache.iceberg.avro.SupportsRowPosition; -import org.apache.iceberg.avro.ValueReader; -import org.apache.iceberg.avro.ValueReaders; -import org.apache.iceberg.data.avro.DecoderResolver; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -/** - * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. - */ -@Deprecated -public class FlinkAvroReader implements DatumReader, SupportsRowPosition { - - private final Schema readSchema; - private final ValueReader reader; - private Schema fileSchema = null; - - /** - * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. - */ - @Deprecated - public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { - this(expectedSchema, readSchema, ImmutableMap.of()); - } - - /** - * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. - */ - @Deprecated - @SuppressWarnings("unchecked") - public FlinkAvroReader( - org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { - this.readSchema = readSchema; - this.reader = - (ValueReader) - AvroSchemaWithTypeVisitor.visit(expectedSchema, readSchema, new ReadBuilder(constants)); - } - - @Override - public void setSchema(Schema newFileSchema) { - this.fileSchema = Schema.applyAliases(newFileSchema, readSchema); - } - - @Override - public RowData read(RowData reuse, Decoder decoder) throws IOException { - return DecoderResolver.resolveAndRead(decoder, readSchema, fileSchema, reader, reuse); - } - - @Override - public void setRowPositionSupplier(Supplier posSupplier) { - if (reader instanceof SupportsRowPosition) { - ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); - } - } - - private static class ReadBuilder extends AvroSchemaWithTypeVisitor> { - private final Map idToConstant; - - private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; - } - - @Override - public ValueReader record( - Types.StructType expected, Schema record, List names, List> fields) { - return FlinkValueReaders.struct(fields, expected.asStructType(), idToConstant); - } - - @Override - public ValueReader union(Type expected, Schema union, List> options) { - return ValueReaders.union(options); - } - - @Override - public ValueReader array( - Types.ListType expected, Schema array, ValueReader elementReader) { - return FlinkValueReaders.array(elementReader); - } - - @Override - public ValueReader map( - Types.MapType expected, Schema map, ValueReader keyReader, ValueReader valueReader) { - return FlinkValueReaders.arrayMap(keyReader, valueReader); - } - - @Override - public ValueReader map(Types.MapType expected, Schema map, ValueReader valueReader) { - return FlinkValueReaders.map(FlinkValueReaders.strings(), valueReader); - } - - @Override - public ValueReader primitive(Type.PrimitiveType expected, Schema primitive) { - LogicalType logicalType = primitive.getLogicalType(); - if (logicalType != null) { - switch (logicalType.getName()) { - case "date": - return ValueReaders.ints(); - - case "time-micros": - return FlinkValueReaders.timeMicros(); - - case "timestamp-millis": - return FlinkValueReaders.timestampMills(); - - case "timestamp-micros": - return FlinkValueReaders.timestampMicros(); - - case "decimal": - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; - return FlinkValueReaders.decimal( - ValueReaders.decimalBytesReader(primitive), - decimal.getPrecision(), - decimal.getScale()); - - case "uuid": - return FlinkValueReaders.uuids(); - - default: - throw new IllegalArgumentException("Unknown logical type: " + logicalType); - } - } - - switch (primitive.getType()) { - case NULL: - return ValueReaders.nulls(); - case BOOLEAN: - return ValueReaders.booleans(); - case INT: - return ValueReaders.ints(); - case LONG: - return ValueReaders.longs(); - case FLOAT: - return ValueReaders.floats(); - case DOUBLE: - return ValueReaders.doubles(); - case STRING: - return FlinkValueReaders.strings(); - case FIXED: - return ValueReaders.fixed(primitive.getFixedSize()); - case BYTES: - return ValueReaders.bytes(); - case ENUM: - return FlinkValueReaders.enums(primitive.getEnumSymbols()); - default: - throw new IllegalArgumentException("Unsupported type: " + primitive); - } - } - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java deleted file mode 100644 index 03910f4fda47..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.data; - -import java.io.File; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; - -/** - * @deprecated should be removed in 1.8.0; along with FlinkAvroReader. - */ -@Deprecated -public class TestFlinkAvroDeprecatedReaderWriter extends AbstractTestFlinkAvroReaderWriter { - - @Override - protected Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema) { - return Avro.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(FlinkAvroReader::new); - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index 282a6055cbd3..a19989836286 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -24,9 +24,6 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; -import java.util.Arrays; -import java.util.List; -import java.util.Map; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; @@ -34,34 +31,20 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.iceberg.Files; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -@ExtendWith(ParameterizedTestExtension.class) public class TestRowProjection { @TempDir private Path temp; - @Parameter(index = 0) - protected Boolean useAvroPlannedReader; - - @Parameters(name = "useAvroPlannedReader={0}") - protected static List parameters() { - return Arrays.asList(new Object[] {Boolean.FALSE}, new Object[] {Boolean.TRUE}); - } - private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) throws IOException { File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); @@ -78,20 +61,13 @@ private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, Avro.ReadBuilder builder = Avro.read(Files.localInput(file)) .project(readSchema) - .createReaderFunc(FlinkAvroReader::new); - if (useAvroPlannedReader) { - builder = - Avro.read(Files.localInput(file)) - .project(readSchema) - .createResolvingReader(FlinkPlannedAvroReader::create); - } - + .createResolvingReader(FlinkPlannedAvroReader::create); Iterable records = builder.build(); return Iterables.getOnlyElement(records); } - @TestTemplate + @Test public void testFullProjection() throws Exception { Schema schema = new Schema( @@ -106,7 +82,7 @@ public void testFullProjection() throws Exception { assertThat(projected.getString(1)).asString().isEqualTo("test"); } - @TestTemplate + @Test public void testSpecialCharacterProjection() throws Exception { Schema schema = new Schema( @@ -126,7 +102,7 @@ public void testSpecialCharacterProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @TestTemplate + @Test public void testReorderedFullProjection() throws Exception { Schema schema = new Schema( @@ -146,7 +122,7 @@ public void testReorderedFullProjection() throws Exception { assertThat(projected.getLong(1)).isEqualTo(34); } - @TestTemplate + @Test public void testReorderedProjection() throws Exception { Schema schema = new Schema( @@ -168,7 +144,7 @@ public void testReorderedProjection() throws Exception { assertThat(projected.isNullAt(2)).isTrue(); } - @TestTemplate + @Test public void testRenamedAddedField() throws Exception { Schema schema = new Schema( @@ -198,7 +174,7 @@ public void testRenamedAddedField() throws Exception { assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); } - @TestTemplate + @Test public void testEmptyProjection() throws Exception { Schema schema = new Schema( @@ -213,7 +189,7 @@ public void testEmptyProjection() throws Exception { assertThat(projected.getArity()).isEqualTo(0); } - @TestTemplate + @Test public void testBasicProjection() throws Exception { Schema writeSchema = new Schema( @@ -236,7 +212,7 @@ public void testBasicProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @TestTemplate + @Test public void testRename() throws Exception { Schema writeSchema = new Schema( @@ -259,7 +235,7 @@ public void testRename() throws Exception { .isEqualTo("test"); } - @TestTemplate + @Test public void testNestedStructProjection() throws Exception { Schema writeSchema = new Schema( @@ -325,7 +301,7 @@ public void testNestedStructProjection() throws Exception { .isEqualTo(-1.539054f, withPrecision(0.000001f)); } - @TestTemplate + @Test public void testMapProjection() throws IOException { Schema writeSchema = new Schema( @@ -367,19 +343,7 @@ public void testMapProjection() throws IOException { assertThat(projected.getMap(0)).isEqualTo(properties); } - private Map toStringMap(Map map) { - Map stringMap = Maps.newHashMap(); - for (Map.Entry entry : map.entrySet()) { - if (entry.getValue() instanceof CharSequence) { - stringMap.put(entry.getKey().toString(), entry.getValue().toString()); - } else { - stringMap.put(entry.getKey().toString(), entry.getValue()); - } - } - return stringMap; - } - - @TestTemplate + @Test public void testMapOfStructsProjection() throws IOException { Schema writeSchema = new Schema( @@ -479,7 +443,7 @@ public void testMapOfStructsProjection() throws IOException { .isEqualTo(52.995143f, withPrecision(0.000001f)); } - @TestTemplate + @Test public void testListProjection() throws IOException { Schema writeSchema = new Schema( @@ -508,8 +472,7 @@ public void testListProjection() throws IOException { assertThat(projected.getArray(0)).isEqualTo(values); } - @TestTemplate - @SuppressWarnings("unchecked") + @Test public void testListOfStructsProjection() throws IOException { Schema writeSchema = new Schema( @@ -585,7 +548,7 @@ public void testListOfStructsProjection() throws IOException { assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); } - @TestTemplate + @Test public void testAddedFieldsWithRequiredChildren() throws Exception { Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java deleted file mode 100644 index baae91dd1882..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.data; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.Decoder; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.avro.AvroSchemaWithTypeVisitor; -import org.apache.iceberg.avro.SupportsRowPosition; -import org.apache.iceberg.avro.ValueReader; -import org.apache.iceberg.avro.ValueReaders; -import org.apache.iceberg.data.avro.DecoderResolver; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -/** - * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. - */ -@Deprecated -public class FlinkAvroReader implements DatumReader, SupportsRowPosition { - - private final Schema readSchema; - private final ValueReader reader; - private Schema fileSchema = null; - - /** - * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. - */ - @Deprecated - public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { - this(expectedSchema, readSchema, ImmutableMap.of()); - } - - /** - * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. - */ - @Deprecated - @SuppressWarnings("unchecked") - public FlinkAvroReader( - org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { - this.readSchema = readSchema; - this.reader = - (ValueReader) - AvroSchemaWithTypeVisitor.visit(expectedSchema, readSchema, new ReadBuilder(constants)); - } - - @Override - public void setSchema(Schema newFileSchema) { - this.fileSchema = Schema.applyAliases(newFileSchema, readSchema); - } - - @Override - public RowData read(RowData reuse, Decoder decoder) throws IOException { - return DecoderResolver.resolveAndRead(decoder, readSchema, fileSchema, reader, reuse); - } - - @Override - public void setRowPositionSupplier(Supplier posSupplier) { - if (reader instanceof SupportsRowPosition) { - ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); - } - } - - private static class ReadBuilder extends AvroSchemaWithTypeVisitor> { - private final Map idToConstant; - - private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; - } - - @Override - public ValueReader record( - Types.StructType expected, Schema record, List names, List> fields) { - return FlinkValueReaders.struct(fields, expected.asStructType(), idToConstant); - } - - @Override - public ValueReader union(Type expected, Schema union, List> options) { - return ValueReaders.union(options); - } - - @Override - public ValueReader array( - Types.ListType expected, Schema array, ValueReader elementReader) { - return FlinkValueReaders.array(elementReader); - } - - @Override - public ValueReader map( - Types.MapType expected, Schema map, ValueReader keyReader, ValueReader valueReader) { - return FlinkValueReaders.arrayMap(keyReader, valueReader); - } - - @Override - public ValueReader map(Types.MapType expected, Schema map, ValueReader valueReader) { - return FlinkValueReaders.map(FlinkValueReaders.strings(), valueReader); - } - - @Override - public ValueReader primitive(Type.PrimitiveType expected, Schema primitive) { - LogicalType logicalType = primitive.getLogicalType(); - if (logicalType != null) { - switch (logicalType.getName()) { - case "date": - return ValueReaders.ints(); - - case "time-micros": - return FlinkValueReaders.timeMicros(); - - case "timestamp-millis": - return FlinkValueReaders.timestampMills(); - - case "timestamp-micros": - return FlinkValueReaders.timestampMicros(); - - case "decimal": - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; - return FlinkValueReaders.decimal( - ValueReaders.decimalBytesReader(primitive), - decimal.getPrecision(), - decimal.getScale()); - - case "uuid": - return FlinkValueReaders.uuids(); - - default: - throw new IllegalArgumentException("Unknown logical type: " + logicalType); - } - } - - switch (primitive.getType()) { - case NULL: - return ValueReaders.nulls(); - case BOOLEAN: - return ValueReaders.booleans(); - case INT: - return ValueReaders.ints(); - case LONG: - return ValueReaders.longs(); - case FLOAT: - return ValueReaders.floats(); - case DOUBLE: - return ValueReaders.doubles(); - case STRING: - return FlinkValueReaders.strings(); - case FIXED: - return ValueReaders.fixed(primitive.getFixedSize()); - case BYTES: - return ValueReaders.bytes(); - case ENUM: - return FlinkValueReaders.enums(primitive.getEnumSymbols()); - default: - throw new IllegalArgumentException("Unsupported type: " + primitive); - } - } - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java deleted file mode 100644 index 03910f4fda47..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.data; - -import java.io.File; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; - -/** - * @deprecated should be removed in 1.8.0; along with FlinkAvroReader. - */ -@Deprecated -public class TestFlinkAvroDeprecatedReaderWriter extends AbstractTestFlinkAvroReaderWriter { - - @Override - protected Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema) { - return Avro.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(FlinkAvroReader::new); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index f76e4c4942bd..004cc8234876 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -24,9 +24,6 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; -import java.util.Arrays; -import java.util.List; -import java.util.Map; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; @@ -34,35 +31,21 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.iceberg.Files; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -@ExtendWith(ParameterizedTestExtension.class) public class TestRowProjection { @TempDir private Path temp; - @Parameter(index = 0) - protected Boolean useAvroPlannedReader; - - @Parameters(name = "useAvroPlannedReader={0}") - protected static List parameters() { - return Arrays.asList(new Object[] {Boolean.FALSE}, new Object[] {Boolean.TRUE}); - } - private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) throws IOException { File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); @@ -79,20 +62,14 @@ private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, Avro.ReadBuilder builder = Avro.read(Files.localInput(file)) .project(readSchema) - .createReaderFunc(FlinkAvroReader::new); - if (useAvroPlannedReader) { - builder = - Avro.read(Files.localInput(file)) - .project(readSchema) - .createResolvingReader(FlinkPlannedAvroReader::create); - } + .createResolvingReader(FlinkPlannedAvroReader::create); Iterable records = builder.build(); return Iterables.getOnlyElement(records); } - @TestTemplate + @Test public void testFullProjection() throws Exception { Schema schema = new Schema( @@ -107,7 +84,7 @@ public void testFullProjection() throws Exception { assertThat(projected.getString(1)).asString().isEqualTo("test"); } - @TestTemplate + @Test public void testSpecialCharacterProjection() throws Exception { Schema schema = new Schema( @@ -127,7 +104,7 @@ public void testSpecialCharacterProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @TestTemplate + @Test public void testReorderedFullProjection() throws Exception { Schema schema = new Schema( @@ -147,7 +124,7 @@ public void testReorderedFullProjection() throws Exception { assertThat(projected.getLong(1)).isEqualTo(34); } - @TestTemplate + @Test public void testReorderedProjection() throws Exception { Schema schema = new Schema( @@ -169,7 +146,7 @@ public void testReorderedProjection() throws Exception { assertThat(projected.isNullAt(2)).isTrue(); } - @TestTemplate + @Test public void testRenamedAddedField() throws Exception { Schema schema = new Schema( @@ -199,7 +176,7 @@ public void testRenamedAddedField() throws Exception { assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); } - @TestTemplate + @Test public void testEmptyProjection() throws Exception { Schema schema = new Schema( @@ -214,7 +191,7 @@ public void testEmptyProjection() throws Exception { assertThat(projected.getArity()).isEqualTo(0); } - @TestTemplate + @Test public void testBasicProjection() throws Exception { Schema writeSchema = new Schema( @@ -238,7 +215,7 @@ public void testBasicProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @TestTemplate + @Test public void testRename() throws Exception { Schema writeSchema = new Schema( @@ -261,7 +238,7 @@ public void testRename() throws Exception { .isEqualTo("test"); } - @TestTemplate + @Test public void testNestedStructProjection() throws Exception { Schema writeSchema = new Schema( @@ -327,7 +304,7 @@ public void testNestedStructProjection() throws Exception { .isEqualTo(-1.539054f, withPrecision(0.000001f)); } - @TestTemplate + @Test public void testMapProjection() throws IOException { Schema writeSchema = new Schema( @@ -369,19 +346,7 @@ public void testMapProjection() throws IOException { assertThat(projected.getMap(0)).isEqualTo(properties); } - private Map toStringMap(Map map) { - Map stringMap = Maps.newHashMap(); - for (Map.Entry entry : map.entrySet()) { - if (entry.getValue() instanceof CharSequence) { - stringMap.put(entry.getKey().toString(), entry.getValue().toString()); - } else { - stringMap.put(entry.getKey().toString(), entry.getValue()); - } - } - return stringMap; - } - - @TestTemplate + @Test public void testMapOfStructsProjection() throws IOException { Schema writeSchema = new Schema( @@ -481,7 +446,7 @@ public void testMapOfStructsProjection() throws IOException { .isEqualTo(52.995143f, withPrecision(0.000001f)); } - @TestTemplate + @Test public void testListProjection() throws IOException { Schema writeSchema = new Schema( @@ -510,7 +475,7 @@ public void testListProjection() throws IOException { assertThat(projected.getArray(0)).isEqualTo(values); } - @TestTemplate + @Test @SuppressWarnings("unchecked") public void testListOfStructsProjection() throws IOException { Schema writeSchema = @@ -587,7 +552,7 @@ public void testListOfStructsProjection() throws IOException { assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); } - @TestTemplate + @Test public void testAddedFieldsWithRequiredChildren() throws Exception { Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java deleted file mode 100644 index baae91dd1882..000000000000 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.data; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.Decoder; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.avro.AvroSchemaWithTypeVisitor; -import org.apache.iceberg.avro.SupportsRowPosition; -import org.apache.iceberg.avro.ValueReader; -import org.apache.iceberg.avro.ValueReaders; -import org.apache.iceberg.data.avro.DecoderResolver; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -/** - * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. - */ -@Deprecated -public class FlinkAvroReader implements DatumReader, SupportsRowPosition { - - private final Schema readSchema; - private final ValueReader reader; - private Schema fileSchema = null; - - /** - * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. - */ - @Deprecated - public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { - this(expectedSchema, readSchema, ImmutableMap.of()); - } - - /** - * @deprecated will be removed in 1.8.0; use FlinkPlannedAvroReader instead. - */ - @Deprecated - @SuppressWarnings("unchecked") - public FlinkAvroReader( - org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { - this.readSchema = readSchema; - this.reader = - (ValueReader) - AvroSchemaWithTypeVisitor.visit(expectedSchema, readSchema, new ReadBuilder(constants)); - } - - @Override - public void setSchema(Schema newFileSchema) { - this.fileSchema = Schema.applyAliases(newFileSchema, readSchema); - } - - @Override - public RowData read(RowData reuse, Decoder decoder) throws IOException { - return DecoderResolver.resolveAndRead(decoder, readSchema, fileSchema, reader, reuse); - } - - @Override - public void setRowPositionSupplier(Supplier posSupplier) { - if (reader instanceof SupportsRowPosition) { - ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); - } - } - - private static class ReadBuilder extends AvroSchemaWithTypeVisitor> { - private final Map idToConstant; - - private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; - } - - @Override - public ValueReader record( - Types.StructType expected, Schema record, List names, List> fields) { - return FlinkValueReaders.struct(fields, expected.asStructType(), idToConstant); - } - - @Override - public ValueReader union(Type expected, Schema union, List> options) { - return ValueReaders.union(options); - } - - @Override - public ValueReader array( - Types.ListType expected, Schema array, ValueReader elementReader) { - return FlinkValueReaders.array(elementReader); - } - - @Override - public ValueReader map( - Types.MapType expected, Schema map, ValueReader keyReader, ValueReader valueReader) { - return FlinkValueReaders.arrayMap(keyReader, valueReader); - } - - @Override - public ValueReader map(Types.MapType expected, Schema map, ValueReader valueReader) { - return FlinkValueReaders.map(FlinkValueReaders.strings(), valueReader); - } - - @Override - public ValueReader primitive(Type.PrimitiveType expected, Schema primitive) { - LogicalType logicalType = primitive.getLogicalType(); - if (logicalType != null) { - switch (logicalType.getName()) { - case "date": - return ValueReaders.ints(); - - case "time-micros": - return FlinkValueReaders.timeMicros(); - - case "timestamp-millis": - return FlinkValueReaders.timestampMills(); - - case "timestamp-micros": - return FlinkValueReaders.timestampMicros(); - - case "decimal": - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; - return FlinkValueReaders.decimal( - ValueReaders.decimalBytesReader(primitive), - decimal.getPrecision(), - decimal.getScale()); - - case "uuid": - return FlinkValueReaders.uuids(); - - default: - throw new IllegalArgumentException("Unknown logical type: " + logicalType); - } - } - - switch (primitive.getType()) { - case NULL: - return ValueReaders.nulls(); - case BOOLEAN: - return ValueReaders.booleans(); - case INT: - return ValueReaders.ints(); - case LONG: - return ValueReaders.longs(); - case FLOAT: - return ValueReaders.floats(); - case DOUBLE: - return ValueReaders.doubles(); - case STRING: - return FlinkValueReaders.strings(); - case FIXED: - return ValueReaders.fixed(primitive.getFixedSize()); - case BYTES: - return ValueReaders.bytes(); - case ENUM: - return FlinkValueReaders.enums(primitive.getEnumSymbols()); - default: - throw new IllegalArgumentException("Unsupported type: " + primitive); - } - } - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java deleted file mode 100644 index 03910f4fda47..000000000000 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroDeprecatedReaderWriter.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.data; - -import java.io.File; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; - -/** - * @deprecated should be removed in 1.8.0; along with FlinkAvroReader. - */ -@Deprecated -public class TestFlinkAvroDeprecatedReaderWriter extends AbstractTestFlinkAvroReaderWriter { - - @Override - protected Avro.ReadBuilder createAvroReadBuilder(File recordsFile, Schema schema) { - return Avro.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(FlinkAvroReader::new); - } -} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index f76e4c4942bd..3b98939ea167 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -24,9 +24,6 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; -import java.util.Arrays; -import java.util.List; -import java.util.Map; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; @@ -34,35 +31,21 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.iceberg.Files; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -@ExtendWith(ParameterizedTestExtension.class) public class TestRowProjection { @TempDir private Path temp; - @Parameter(index = 0) - protected Boolean useAvroPlannedReader; - - @Parameters(name = "useAvroPlannedReader={0}") - protected static List parameters() { - return Arrays.asList(new Object[] {Boolean.FALSE}, new Object[] {Boolean.TRUE}); - } - private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) throws IOException { File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); @@ -79,20 +62,14 @@ private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, Avro.ReadBuilder builder = Avro.read(Files.localInput(file)) .project(readSchema) - .createReaderFunc(FlinkAvroReader::new); - if (useAvroPlannedReader) { - builder = - Avro.read(Files.localInput(file)) - .project(readSchema) - .createResolvingReader(FlinkPlannedAvroReader::create); - } + .createResolvingReader(FlinkPlannedAvroReader::create); Iterable records = builder.build(); return Iterables.getOnlyElement(records); } - @TestTemplate + @Test public void testFullProjection() throws Exception { Schema schema = new Schema( @@ -107,7 +84,7 @@ public void testFullProjection() throws Exception { assertThat(projected.getString(1)).asString().isEqualTo("test"); } - @TestTemplate + @Test public void testSpecialCharacterProjection() throws Exception { Schema schema = new Schema( @@ -127,7 +104,7 @@ public void testSpecialCharacterProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @TestTemplate + @Test public void testReorderedFullProjection() throws Exception { Schema schema = new Schema( @@ -147,7 +124,7 @@ public void testReorderedFullProjection() throws Exception { assertThat(projected.getLong(1)).isEqualTo(34); } - @TestTemplate + @Test public void testReorderedProjection() throws Exception { Schema schema = new Schema( @@ -169,7 +146,7 @@ public void testReorderedProjection() throws Exception { assertThat(projected.isNullAt(2)).isTrue(); } - @TestTemplate + @Test public void testRenamedAddedField() throws Exception { Schema schema = new Schema( @@ -199,7 +176,7 @@ public void testRenamedAddedField() throws Exception { assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); } - @TestTemplate + @Test public void testEmptyProjection() throws Exception { Schema schema = new Schema( @@ -214,7 +191,7 @@ public void testEmptyProjection() throws Exception { assertThat(projected.getArity()).isEqualTo(0); } - @TestTemplate + @Test public void testBasicProjection() throws Exception { Schema writeSchema = new Schema( @@ -238,7 +215,7 @@ public void testBasicProjection() throws Exception { assertThat(projected.getString(0)).asString().isEqualTo("test"); } - @TestTemplate + @Test public void testRename() throws Exception { Schema writeSchema = new Schema( @@ -261,7 +238,7 @@ public void testRename() throws Exception { .isEqualTo("test"); } - @TestTemplate + @Test public void testNestedStructProjection() throws Exception { Schema writeSchema = new Schema( @@ -327,7 +304,7 @@ public void testNestedStructProjection() throws Exception { .isEqualTo(-1.539054f, withPrecision(0.000001f)); } - @TestTemplate + @Test public void testMapProjection() throws IOException { Schema writeSchema = new Schema( @@ -369,19 +346,7 @@ public void testMapProjection() throws IOException { assertThat(projected.getMap(0)).isEqualTo(properties); } - private Map toStringMap(Map map) { - Map stringMap = Maps.newHashMap(); - for (Map.Entry entry : map.entrySet()) { - if (entry.getValue() instanceof CharSequence) { - stringMap.put(entry.getKey().toString(), entry.getValue().toString()); - } else { - stringMap.put(entry.getKey().toString(), entry.getValue()); - } - } - return stringMap; - } - - @TestTemplate + @Test public void testMapOfStructsProjection() throws IOException { Schema writeSchema = new Schema( @@ -481,7 +446,7 @@ public void testMapOfStructsProjection() throws IOException { .isEqualTo(52.995143f, withPrecision(0.000001f)); } - @TestTemplate + @Test public void testListProjection() throws IOException { Schema writeSchema = new Schema( @@ -510,8 +475,7 @@ public void testListProjection() throws IOException { assertThat(projected.getArray(0)).isEqualTo(values); } - @TestTemplate - @SuppressWarnings("unchecked") + @Test public void testListOfStructsProjection() throws IOException { Schema writeSchema = new Schema( @@ -587,7 +551,7 @@ public void testListOfStructsProjection() throws IOException { assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); } - @TestTemplate + @Test public void testAddedFieldsWithRequiredChildren() throws Exception { Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java deleted file mode 100644 index 7d92d963a9f4..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.data; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.Decoder; -import org.apache.iceberg.avro.AvroSchemaWithTypeVisitor; -import org.apache.iceberg.avro.SupportsRowPosition; -import org.apache.iceberg.avro.ValueReader; -import org.apache.iceberg.avro.ValueReaders; -import org.apache.iceberg.data.avro.DecoderResolver; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.catalyst.InternalRow; - -/** - * @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead. - */ -@Deprecated -public class SparkAvroReader implements DatumReader, SupportsRowPosition { - - private final Schema readSchema; - private final ValueReader reader; - private Schema fileSchema = null; - - /** - * @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead. - */ - @Deprecated - public SparkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { - this(expectedSchema, readSchema, ImmutableMap.of()); - } - - /** - * @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead. - */ - @Deprecated - @SuppressWarnings("unchecked") - public SparkAvroReader( - org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map constants) { - this.readSchema = readSchema; - this.reader = - (ValueReader) - AvroSchemaWithTypeVisitor.visit(expectedSchema, readSchema, new ReadBuilder(constants)); - } - - @Override - public void setSchema(Schema newFileSchema) { - this.fileSchema = Schema.applyAliases(newFileSchema, readSchema); - } - - @Override - public InternalRow read(InternalRow reuse, Decoder decoder) throws IOException { - return DecoderResolver.resolveAndRead(decoder, readSchema, fileSchema, reader, reuse); - } - - @Override - public void setRowPositionSupplier(Supplier posSupplier) { - if (reader instanceof SupportsRowPosition) { - ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); - } - } - - private static class ReadBuilder extends AvroSchemaWithTypeVisitor> { - private final Map idToConstant; - - private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; - } - - @Override - public ValueReader record( - Types.StructType expected, Schema record, List names, List> fields) { - return SparkValueReaders.struct(fields, expected, idToConstant); - } - - @Override - public ValueReader union(Type expected, Schema union, List> options) { - return ValueReaders.union(options); - } - - @Override - public ValueReader array( - Types.ListType expected, Schema array, ValueReader elementReader) { - return SparkValueReaders.array(elementReader); - } - - @Override - public ValueReader map( - Types.MapType expected, Schema map, ValueReader keyReader, ValueReader valueReader) { - return SparkValueReaders.arrayMap(keyReader, valueReader); - } - - @Override - public ValueReader map(Types.MapType expected, Schema map, ValueReader valueReader) { - return SparkValueReaders.map(SparkValueReaders.strings(), valueReader); - } - - @Override - public ValueReader primitive(Type.PrimitiveType expected, Schema primitive) { - LogicalType logicalType = primitive.getLogicalType(); - if (logicalType != null) { - switch (logicalType.getName()) { - case "date": - // Spark uses the same representation - return ValueReaders.ints(); - - case "timestamp-millis": - // adjust to microseconds - ValueReader longs = ValueReaders.longs(); - return (ValueReader) (decoder, ignored) -> longs.read(decoder, null) * 1000L; - - case "timestamp-micros": - // Spark uses the same representation - return ValueReaders.longs(); - - case "decimal": - return SparkValueReaders.decimal( - ValueReaders.decimalBytesReader(primitive), - ((LogicalTypes.Decimal) logicalType).getScale()); - - case "uuid": - return SparkValueReaders.uuids(); - - default: - throw new IllegalArgumentException("Unknown logical type: " + logicalType); - } - } - - switch (primitive.getType()) { - case NULL: - return ValueReaders.nulls(); - case BOOLEAN: - return ValueReaders.booleans(); - case INT: - return ValueReaders.ints(); - case LONG: - return ValueReaders.longs(); - case FLOAT: - return ValueReaders.floats(); - case DOUBLE: - return ValueReaders.doubles(); - case STRING: - return SparkValueReaders.strings(); - case FIXED: - return ValueReaders.fixed(primitive.getFixedSize()); - case BYTES: - return ValueReaders.bytes(); - case ENUM: - return SparkValueReaders.enums(primitive.getEnumSymbols()); - default: - throw new IllegalArgumentException("Unsupported type: " + primitive); - } - } - } -} From ac6509a4e469f808bebb8b713a5c4213f98ff4a5 Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Wed, 11 Dec 2024 05:33:55 +0800 Subject: [PATCH 240/313] Flink: Fix range distribution npe when value is null (#11662) --- .../sink/shuffle/CompletedStatistics.java | 17 +++++ .../CompletedStatisticsSerializer.java | 12 +++ .../shuffle/DataStatisticsCoordinator.java | 3 +- .../flink/sink/shuffle/SortKeySerializer.java | 65 ++++++++++++++-- .../flink/sink/shuffle/StatisticsUtil.java | 25 ++++++- .../TestFlinkIcebergSinkDistributionMode.java | 39 ++++++++++ .../TestCompletedStatisticsSerializer.java | 49 ++++++++++++ .../TestDataStatisticsCoordinator.java | 74 +++++++++++++++++++ .../shuffle/TestDataStatisticsOperator.java | 30 ++++++++ .../TestSortKeySerializerPrimitives.java | 4 +- .../TestSortKeySerializerSnapshot.java | 22 ++++++ 11 files changed, 325 insertions(+), 15 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index e4cba174f0f2..a8bf0f839e49 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -108,4 +108,21 @@ boolean isEmpty() { return keyFrequency().isEmpty(); } } + + boolean isValid() { + if (type == StatisticsType.Sketch) { + if (null == keySamples) { + return false; + } + } else { + if (null == keyFrequency()) { + return false; + } + if (keyFrequency().values().contains(null)) { + return false; + } + } + + return true; + } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java index 1ac0e386a011..48c85a9bd91e 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -48,6 +48,18 @@ class CompletedStatisticsSerializer extends TypeSerializer this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); } + public void changeSortKeySerializerVersion(int version) { + if (sortKeySerializer instanceof SortKeySerializer) { + ((SortKeySerializer) sortKeySerializer).setVersion(version); + } + } + + public void changeSortKeySerializerVersionLatest() { + if (sortKeySerializer instanceof SortKeySerializer) { + ((SortKeySerializer) sortKeySerializer).restoreToLatestVersion(); + } + } + @Override public boolean isImmutableType() { return false; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index e2a282efd82e..4f2afd60fed1 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -370,7 +370,8 @@ public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); this.completedStatistics = StatisticsUtil.deserializeCompletedStatistics( - checkpointData, completedStatisticsSerializer); + checkpointData, (CompletedStatisticsSerializer) completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed this.globalStatistics = globalStatistics( diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index b3e536bdde52..5b979e546d51 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -52,9 +52,12 @@ class SortKeySerializer extends TypeSerializer { private final int size; private final Types.NestedField[] transformedFields; + private int version; + private transient SortKey sortKey; - SortKeySerializer(Schema schema, SortOrder sortOrder) { + SortKeySerializer(Schema schema, SortOrder sortOrder, int version) { + this.version = version; this.schema = schema; this.sortOrder = sortOrder; this.size = sortOrder.fields().size(); @@ -75,6 +78,10 @@ class SortKeySerializer extends TypeSerializer { } } + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this(schema, sortOrder, SortKeySerializerSnapshot.CURRENT_VERSION); + } + private SortKey lazySortKey() { if (sortKey == null) { this.sortKey = new SortKey(schema, sortOrder); @@ -83,6 +90,18 @@ private SortKey lazySortKey() { return sortKey; } + public int getLatestVersion() { + return snapshotConfiguration().getCurrentVersion(); + } + + public void restoreToLatestVersion() { + this.version = snapshotConfiguration().getCurrentVersion(); + } + + public void setVersion(int version) { + this.version = version; + } + @Override public boolean isImmutableType() { return false; @@ -124,6 +143,16 @@ public void serialize(SortKey record, DataOutputView target) throws IOException for (int i = 0; i < size; ++i) { int fieldId = transformedFields[i].fieldId(); Type.TypeID typeId = transformedFields[i].type().typeId(); + if (version > 1) { + Object value = record.get(i, Object.class); + if (value == null) { + target.writeBoolean(true); + continue; + } else { + target.writeBoolean(false); + } + } + switch (typeId) { case BOOLEAN: target.writeBoolean(record.get(i, Boolean.class)); @@ -192,6 +221,14 @@ public SortKey deserialize(SortKey reuse, DataInputView source) throws IOExcepti reuse.size(), size); for (int i = 0; i < size; ++i) { + if (version > 1) { + boolean isNull = source.readBoolean(); + if (isNull) { + reuse.set(i, null); + continue; + } + } + int fieldId = transformedFields[i].fieldId(); Type.TypeID typeId = transformedFields[i].type().typeId(); switch (typeId) { @@ -276,11 +313,13 @@ public TypeSerializerSnapshot snapshotConfiguration() { } public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { - private static final int CURRENT_VERSION = 1; + private static final int CURRENT_VERSION = 2; private Schema schema; private SortOrder sortOrder; + private int version = CURRENT_VERSION; + /** Constructor for read instantiation. */ @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public SortKeySerializerSnapshot() { @@ -310,10 +349,16 @@ public void writeSnapshot(DataOutputView out) throws IOException { @Override public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { - if (readVersion == 1) { - readV1(in); - } else { - throw new IllegalArgumentException("Unknown read version: " + readVersion); + switch (readVersion) { + case 1: + read(in); + this.version = 1; + break; + case 2: + read(in); + break; + default: + throw new IllegalArgumentException("Unknown read version: " + readVersion); } } @@ -324,6 +369,10 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } + if (oldSerializerSnapshot.getCurrentVersion() == 1 && this.getCurrentVersion() == 2) { + return TypeSerializerSchemaCompatibility.compatibleAfterMigration(); + } + // Sort order should be identical SortKeySerializerSnapshot oldSnapshot = (SortKeySerializerSnapshot) oldSerializerSnapshot; if (!sortOrder.sameOrder(oldSnapshot.sortOrder)) { @@ -349,10 +398,10 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( public TypeSerializer restoreSerializer() { Preconditions.checkState(schema != null, "Invalid schema: null"); Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); - return new SortKeySerializer(schema, sortOrder); + return new SortKeySerializer(schema, sortOrder, version); } - private void readV1(DataInputView in) throws IOException { + private void read(DataInputView in) throws IOException { String schemaJson = StringUtils.readString(in); String sortOrderJson = StringUtils.readString(in); this.schema = SchemaParser.fromJson(schemaJson); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java index 5d48ec57ca49..f2efc7fa9834 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java @@ -73,12 +73,29 @@ static byte[] serializeCompletedStatistics( } static CompletedStatistics deserializeCompletedStatistics( - byte[] bytes, TypeSerializer statisticsSerializer) { + byte[] bytes, CompletedStatisticsSerializer statisticsSerializer) { try { DataInputDeserializer input = new DataInputDeserializer(bytes); - return statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + CompletedStatistics completedStatistics = statisticsSerializer.deserialize(input); + if (!completedStatistics.isValid()) { + throw new RuntimeException("Fail to deserialize aggregated statistics,change to v1"); + } + + return completedStatistics; + } catch (Exception e) { + try { + // If we restore from a lower version, the new version of SortKeySerializer cannot correctly + // parse the checkpointData, so we need to first switch the version to v1. Once the state + // data is successfully parsed, we need to switch the serialization version to the latest + // version to parse the subsequent data passed from the TM. + statisticsSerializer.changeSortKeySerializerVersion(1); + DataInputDeserializer input = new DataInputDeserializer(bytes); + CompletedStatistics deserialize = statisticsSerializer.deserialize(input); + statisticsSerializer.changeSortKeySerializerVersionLatest(); + return deserialize; + } catch (IOException ioException) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", ioException); + } } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java index df8c3c79d3e3..aa9a0291b38f 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -46,6 +46,7 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; 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.Maps; @@ -252,6 +253,44 @@ public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); } + @TestTemplate + public void testRangeDistributionWithNullValue() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + List> charRows = createCharRows(numOfCheckpoints, 10); + charRows.add(ImmutableList.of(Row.of(1, null))); + DataStream dataStream = + env.addSource(createRangeDistributionBoundedSource(charRows), ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // sort based on partition columns + builder.append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + } + @TestTemplate public void testRangeDistributionWithSortOrder() throws Exception { table diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java index 4ee9888934a8..1975d7e8d654 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -19,11 +19,15 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; public class TestCompletedStatisticsSerializer extends SerializerTestBase { @@ -51,4 +55,49 @@ protected CompletedStatistics[] getTestData() { CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) }; } + + @Test + public void testSerializer() throws Exception { + TypeSerializer completedStatisticsTypeSerializer = createSerializer(); + CompletedStatistics[] data = getTestData(); + DataOutputSerializer output = new DataOutputSerializer(1024); + completedStatisticsTypeSerializer.serialize(data[0], output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + CompletedStatistics deserialized = completedStatisticsTypeSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(data[0]); + } + + @Test + public void testRestoreOldVersionSerializer() throws Exception { + CompletedStatisticsSerializer completedStatisticsTypeSerializer = + (CompletedStatisticsSerializer) createSerializer(); + completedStatisticsTypeSerializer.changeSortKeySerializerVersion(1); + CompletedStatistics[] data = getTestData(); + DataOutputSerializer output = new DataOutputSerializer(1024); + completedStatisticsTypeSerializer.serialize(data[0], output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + completedStatisticsTypeSerializer.changeSortKeySerializerVersionLatest(); + CompletedStatistics completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + serializedBytes, completedStatisticsTypeSerializer); + assertThat(completedStatistics).isEqualTo(data[0]); + } + + @Test + public void testRestoreNewSerializer() throws Exception { + CompletedStatisticsSerializer completedStatisticsTypeSerializer = + (CompletedStatisticsSerializer) createSerializer(); + CompletedStatistics[] data = getTestData(); + DataOutputSerializer output = new DataOutputSerializer(1024); + completedStatisticsTypeSerializer.serialize(data[0], output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + CompletedStatistics completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + serializedBytes, completedStatisticsTypeSerializer); + assertThat(completedStatistics).isEqualTo(data[0]); + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index a08a8a73e80c..acecc5b60af1 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -152,6 +152,80 @@ public void testDataStatisticsEventHandling(StatisticsType type) throws Exceptio } } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testDataStatisticsEventHandlingWithNullValue(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + SortKey nullSortKey = Fixtures.SORT_KEY.copy(); + nullSortKey.set(0, null); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + nullSortKey, + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + nullSortKey, + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + Map keyFrequency = + ImmutableMap.of(nullSortKey, 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 5L); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(NUM_SUBTASKS, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(keyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + nullSortKey, + nullSortKey, + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.checkpointId()).isEqualTo(1L); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + } + } + } + @Test public void testRequestGlobalStatisticsEventHandling() throws Exception { try (DataStatisticsCoordinator dataStatisticsCoordinator = diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index bc248b778184..f7a7a147e73a 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -34,6 +34,8 @@ import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; @@ -136,6 +138,34 @@ public void testProcessElement(StatisticsType type) throws Exception { } } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testProcessElementWithNull(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + operator.processElement(new StreamRecord<>(GenericRowData.of(null, 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + + DataStatistics localStatistics = operator.localStatistics(); + SortKeySerializer sortKeySerializer = + new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); + DataStatisticsSerializer taskStatisticsSerializer = + new DataStatisticsSerializer(sortKeySerializer); + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + taskStatisticsSerializer.serialize(localStatistics, outputView); + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + DataStatistics dataStatistics = taskStatisticsSerializer.deserialize(inputView); + + testHarness.endInput(); + + assertThat(localStatistics).isEqualTo(dataStatistics); + } + } + @ParameterizedTest @EnumSource(StatisticsType.class) public void testOperatorOutput(StatisticsType type) throws Exception { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java index 54cceae6e55b..ac2e2784e681 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -80,8 +80,8 @@ public void testSerializationSize() throws Exception { byte[] serializedBytes = output.getCopyOfBuffer(); assertThat(serializedBytes.length) .as( - "Serialized bytes for sort key should be 38 bytes (34 UUID text + 4 byte integer of string length") - .isEqualTo(38); + "Serialized bytes for sort key should be 39 bytes (34 UUID text + 4 byte integer of string length + 1 byte of isnull flag") + .isEqualTo(39); DataInputDeserializer input = new DataInputDeserializer(serializedBytes); SortKey deserialized = serializer.deserialize(input); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java index c0f688f2589e..2d87b089cecb 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -73,6 +73,28 @@ public void testRestoredSerializer() throws Exception { assertThat(deserialized).isEqualTo(sortKey); } + @Test + public void testRestoredOldSerializer() throws Exception { + RowData rowData = GenericRowData.of(StringData.fromString("str"), 1); + RowDataWrapper rowDataWrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(struct); + + SortKeySerializer originalSerializer = new SortKeySerializer(SCHEMA, SORT_ORDER, 1); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + ((SortKeySerializer) restoredSerializer).setVersion(1); + DataOutputSerializer output = new DataOutputSerializer(1024); + originalSerializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = restoredSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } + @Test public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = From ff813445916bfd6ec1cc30a02b02f8bade7a26f6 Mon Sep 17 00:00:00 2001 From: hsiang-c <137842490+hsiang-c@users.noreply.github.com> Date: Wed, 11 Dec 2024 09:20:12 +0800 Subject: [PATCH 241/313] AWS: Enable RetryMode for AWS KMS client (#11420) --- .../aws/AssumeRoleAwsClientFactory.java | 7 ++++ .../iceberg/aws/AwsClientFactories.java | 1 + .../iceberg/aws/AwsClientProperties.java | 22 ++++++++++ .../LakeFormationAwsClientFactory.java | 1 + .../aws/kms/TestKmsClientProperties.java | 40 +++++++++++++++++++ 5 files changed, 71 insertions(+) create mode 100644 aws/src/test/java/org/apache/iceberg/aws/kms/TestKmsClientProperties.java diff --git a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java index 4c3c305d4bae..4149d795d30d 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java @@ -38,6 +38,7 @@ public class AssumeRoleAwsClientFactory implements AwsClientFactory { private HttpClientProperties httpClientProperties; private S3FileIOProperties s3FileIOProperties; private String roleSessionName; + private AwsClientProperties awsClientProperties; @Override public S3Client s3() { @@ -64,6 +65,7 @@ public KmsClient kms() { return KmsClient.builder() .applyMutation(this::applyAssumeRoleConfigurations) .applyMutation(httpClientProperties::applyHttpClientConfigurations) + .applyMutation(awsClientProperties::applyRetryConfigurations) .build(); } @@ -81,6 +83,7 @@ public void initialize(Map properties) { this.awsProperties = new AwsProperties(properties); this.s3FileIOProperties = new S3FileIOProperties(properties); this.httpClientProperties = new HttpClientProperties(properties); + this.awsClientProperties = new AwsClientProperties(properties); this.roleSessionName = genSessionName(); Preconditions.checkNotNull( awsProperties.clientAssumeRoleArn(), @@ -126,6 +129,10 @@ protected S3FileIOProperties s3FileIOProperties() { return s3FileIOProperties; } + protected AwsClientProperties awsClientProperties() { + return awsClientProperties; + } + private StsClient sts() { return StsClient.builder() .applyMutation(httpClientProperties::applyHttpClientConfigurations) diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java index 5974e21209e3..7554b5629be4 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -134,6 +134,7 @@ public KmsClient kms() { .applyMutation(awsClientProperties::applyClientRegionConfiguration) .applyMutation(httpClientProperties::applyHttpClientConfigurations) .applyMutation(awsClientProperties::applyClientCredentialConfigurations) + .applyMutation(awsClientProperties::applyRetryConfigurations) .build(); } diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java index 4f2d4d6a5a6c..abdea1ae9fcd 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java @@ -32,6 +32,8 @@ import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryMode; import software.amazon.awssdk.regions.Region; public class AwsClientProperties implements Serializable { @@ -178,6 +180,26 @@ public AwsCredentialsProvider credentialsProvider( return DefaultCredentialsProvider.builder().build(); } + /** + * Configure RetryMode + * to ADAPTIVE_V2 for AWS clients + * + *

    Sample usage: + * + *

    +   *   KmsClient.builder().applyMutation(awsClientProperties::applyRetryConfigurations)
    +   * 
    + */ + public void applyRetryConfigurations(T builder) { + ClientOverrideConfiguration.Builder configBuilder = + null != builder.overrideConfiguration() + ? builder.overrideConfiguration().toBuilder() + : ClientOverrideConfiguration.builder(); + + builder.overrideConfiguration(configBuilder.retryStrategy(RetryMode.ADAPTIVE_V2).build()); + } + private AwsCredentialsProvider credentialsProvider(String credentialsProviderClass) { Class providerClass; try { diff --git a/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java index 5d37470066d2..aaab2c2bc769 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java @@ -96,6 +96,7 @@ public KmsClient kms() { if (isTableRegisteredWithLakeFormation()) { return KmsClient.builder() .applyMutation(httpClientProperties()::applyHttpClientConfigurations) + .applyMutation(awsClientProperties()::applyRetryConfigurations) .credentialsProvider( new LakeFormationCredentialsProvider(lakeFormation(), buildTableArn())) .region(Region.of(region())) diff --git a/aws/src/test/java/org/apache/iceberg/aws/kms/TestKmsClientProperties.java b/aws/src/test/java/org/apache/iceberg/aws/kms/TestKmsClientProperties.java new file mode 100644 index 000000000000..9d1fb8db51eb --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/kms/TestKmsClientProperties.java @@ -0,0 +1,40 @@ +/* + * 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.aws.kms; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.aws.AwsClientProperties; +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.core.retry.RetryMode; +import software.amazon.awssdk.services.kms.KmsClient; +import software.amazon.awssdk.services.kms.KmsClientBuilder; + +public class TestKmsClientProperties { + @Test + public void testApplyRetryConfiguration() { + AwsClientProperties awsClientProperties = new AwsClientProperties(); + + KmsClientBuilder builder = KmsClient.builder(); + awsClientProperties.applyRetryConfigurations(builder); + RetryMode retryPolicy = builder.overrideConfiguration().retryMode().get(); + + assertThat(retryPolicy).as("retry mode should be ADAPTIVE_V2").isEqualTo(RetryMode.ADAPTIVE_V2); + } +} From da53495bc1bb52db37cdd1ced5c2377001c9d482 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Wed, 11 Dec 2024 10:04:59 -0700 Subject: [PATCH 242/313] Core, Flink, Spark, KafkaConnect: Remove usage of deprecated path API (#11744) --- .../org/apache/iceberg/SnapshotProducer.java | 2 +- .../java/org/apache/iceberg/V1Metadata.java | 2 +- .../java/org/apache/iceberg/V2Metadata.java | 2 +- .../java/org/apache/iceberg/V3Metadata.java | 2 +- .../apache/iceberg/TestRewriteManifests.java | 2 +- .../data/TestDataFileIndexStatsFilters.java | 2 +- .../io/TestTaskEqualityDeltaWriter.java | 34 ++--- .../source/RowDataFileScanTaskReader.java | 2 +- .../source/split/IcebergSourceSplit.java | 2 +- ...TestBucketPartitionerFlinkIcebergSink.java | 2 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 2 +- .../iceberg/flink/sink/TestTaskWriters.java | 6 +- .../TestContinuousSplitPlannerImpl.java | 28 ++-- .../source/RowDataFileScanTaskReader.java | 2 +- .../source/split/IcebergSourceSplit.java | 2 +- .../iceberg/flink/TestFlinkCatalogTable.java | 4 +- .../org/apache/iceberg/flink/TestHelpers.java | 2 +- .../actions/TestRewriteDataFilesAction.java | 2 +- ...TestBucketPartitionerFlinkIcebergSink.java | 2 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 2 +- .../iceberg/flink/sink/TestIcebergSinkV2.java | 2 +- .../iceberg/flink/sink/TestTaskWriters.java | 6 +- .../TestContinuousSplitPlannerImpl.java | 31 +++-- .../TestIcebergSourceSplitSerializer.java | 2 +- .../source/RowDataFileScanTaskReader.java | 2 +- .../source/split/IcebergSourceSplit.java | 2 +- .../iceberg/flink/TestFlinkCatalogTable.java | 4 +- .../org/apache/iceberg/flink/TestHelpers.java | 2 +- .../actions/TestRewriteDataFilesAction.java | 2 +- ...TestBucketPartitionerFlinkIcebergSink.java | 2 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 2 +- .../iceberg/flink/sink/TestIcebergSinkV2.java | 2 +- .../iceberg/flink/sink/TestTaskWriters.java | 6 +- .../TestContinuousSplitPlannerImpl.java | 31 +++-- .../TestIcebergSourceSplitSerializer.java | 2 +- .../apache/iceberg/hive/HiveTableTest.java | 4 +- .../iceberg/connect/channel/Coordinator.java | 5 +- .../mr/hive/HiveIcebergOutputCommitter.java | 6 +- .../mr/hive/HiveIcebergRecordWriter.java | 2 +- .../mr/mapreduce/IcebergInputFormat.java | 5 +- .../apache/iceberg/orc/TestOrcDataWriter.java | 2 +- .../TestExpireSnapshotsProcedure.java | 2 +- .../TestRemoveOrphanFilesProcedure.java | 3 +- .../TestRewritePositionDeleteFiles.java | 2 +- .../iceberg/spark/extensions/TestUpdate.java | 2 +- ...SourceParquetMultiDeleteFileBenchmark.java | 2 +- ...cebergSourceParquetPosDeleteBenchmark.java | 2 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 2 +- .../apache/iceberg/spark/SparkTableUtil.java | 7 +- .../spark/actions/BaseSparkAction.java | 2 +- .../iceberg/spark/source/BaseReader.java | 4 +- .../iceberg/spark/source/BatchDataReader.java | 2 +- .../spark/source/ChangelogRowReader.java | 6 +- .../spark/source/EqualityDeleteRowReader.java | 4 +- .../source/PositionDeletesRowReader.java | 4 +- .../iceberg/spark/source/RowDataReader.java | 4 +- .../spark/source/SparkCleanupUtil.java | 2 +- .../spark/source/SparkCopyOnWriteScan.java | 2 +- .../org/apache/iceberg/TaskCheckHelper.java | 5 +- .../org/apache/iceberg/ValidationHelpers.java | 4 +- .../apache/iceberg/spark/SparkTestBase.java | 2 +- .../TestDeleteReachableFilesAction.java | 2 +- .../actions/TestExpireSnapshotsAction.java | 38 +++--- .../actions/TestRewriteDataFilesAction.java | 3 +- .../TestRewritePositionDeleteFilesAction.java | 12 +- .../iceberg/spark/source/TestBaseReader.java | 2 +- .../spark/source/TestDataFrameWrites.java | 4 +- .../spark/source/TestDataSourceOptions.java | 4 +- .../source/TestIcebergSourceTablesBase.java | 8 +- .../source/TestPositionDeletesTable.java | 74 +++++------ .../spark/source/TestRuntimeFiltering.java | 4 +- .../spark/source/TestSparkDataFile.java | 2 +- .../spark/source/TestSparkReaderDeletes.java | 42 +++--- .../TestExpireSnapshotsProcedure.java | 2 +- .../TestRemoveOrphanFilesProcedure.java | 3 +- .../TestRewritePositionDeleteFiles.java | 2 +- .../extensions/TestSparkExecutorCache.java | 4 +- .../iceberg/spark/extensions/TestUpdate.java | 2 +- ...SourceParquetMultiDeleteFileBenchmark.java | 2 +- ...cebergSourceParquetPosDeleteBenchmark.java | 2 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 2 +- .../apache/iceberg/spark/SparkTableUtil.java | 7 +- .../spark/actions/BaseSparkAction.java | 2 +- .../RemoveDanglingDeletesSparkAction.java | 2 +- .../iceberg/spark/source/BaseReader.java | 4 +- .../iceberg/spark/source/BatchDataReader.java | 2 +- .../spark/source/ChangelogRowReader.java | 6 +- .../spark/source/EqualityDeleteRowReader.java | 4 +- .../source/PositionDeletesRowReader.java | 4 +- .../iceberg/spark/source/RowDataReader.java | 4 +- .../spark/source/SparkCleanupUtil.java | 2 +- .../spark/source/SparkCopyOnWriteScan.java | 2 +- .../org/apache/iceberg/TaskCheckHelper.java | 5 +- .../org/apache/iceberg/ValidationHelpers.java | 4 +- .../apache/iceberg/spark/SparkTestBase.java | 2 +- .../iceberg/spark/TestSparkExecutorCache.java | 4 +- .../TestDeleteReachableFilesAction.java | 2 +- .../actions/TestExpireSnapshotsAction.java | 38 +++--- .../TestRemoveDanglingDeleteAction.java | 122 +++++++++--------- .../actions/TestRewriteDataFilesAction.java | 3 +- .../TestRewritePositionDeleteFilesAction.java | 12 +- .../iceberg/spark/source/TestBaseReader.java | 2 +- .../spark/source/TestCompressionSettings.java | 6 +- .../spark/source/TestDataFrameWrites.java | 4 +- .../spark/source/TestDataSourceOptions.java | 4 +- .../source/TestIcebergSourceTablesBase.java | 8 +- .../source/TestPositionDeletesTable.java | 74 +++++------ .../spark/source/TestRuntimeFiltering.java | 4 +- .../spark/source/TestSparkDataFile.java | 6 +- .../spark/source/TestSparkReaderDeletes.java | 42 +++--- .../TestExpireSnapshotsProcedure.java | 2 +- .../TestRemoveOrphanFilesProcedure.java | 3 +- ...SourceParquetMultiDeleteFileBenchmark.java | 2 +- ...cebergSourceParquetPosDeleteBenchmark.java | 2 +- ...ceParquetWithUnrelatedDeleteBenchmark.java | 2 +- .../apache/iceberg/spark/SparkTableUtil.java | 7 +- .../spark/source/ChangelogRowReader.java | 4 +- .../org/apache/iceberg/ValidationHelpers.java | 2 +- .../org/apache/iceberg/spark/TestBase.java | 2 +- .../iceberg/spark/TestSparkExecutorCache.java | 2 +- .../actions/TestExpireSnapshotsAction.java | 6 +- .../TestRemoveDanglingDeleteAction.java | 122 +++++++++--------- .../actions/TestRewriteDataFilesAction.java | 3 +- .../TestRewritePositionDeleteFilesAction.java | 12 +- .../source/TestIcebergSourceTablesBase.java | 8 +- .../source/TestPositionDeletesTable.java | 5 +- 126 files changed, 535 insertions(+), 533 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index f5a7e99b684e..bc65e90eaeae 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -850,7 +850,7 @@ public FileContent content() { @Override public CharSequence path() { - return deleteFile.path(); + return deleteFile.location(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/V1Metadata.java b/core/src/main/java/org/apache/iceberg/V1Metadata.java index c5319a0bc100..81fd65d99803 100644 --- a/core/src/main/java/org/apache/iceberg/V1Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V1Metadata.java @@ -416,7 +416,7 @@ public FileContent content() { @Override public CharSequence path() { - return wrapped.path(); + return wrapped.location(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index 18c3b0a40613..2b98cd767c94 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -486,7 +486,7 @@ public FileContent content() { @Override public CharSequence path() { - return wrapped.path(); + return wrapped.location(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java index fc11a7df03de..458796d4752a 100644 --- a/core/src/main/java/org/apache/iceberg/V3Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -500,7 +500,7 @@ public FileContent content() { @Override public CharSequence path() { - return wrapped.path(); + return wrapped.location(); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index e96cd5c0cccf..532f51e3589d 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -164,7 +164,7 @@ public void testReplaceManifestsSeparate() { // cluster by path will split the manifest into two - table.rewriteManifests().clusterBy(file -> file.location()).commit(); + table.rewriteManifests().clusterBy(ContentFile::location).commit(); List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).hasSize(2); diff --git a/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java b/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java index 2cbf2ad4d1f0..5de7cca92426 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java +++ b/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java @@ -451,7 +451,7 @@ private void assertDeletes(FileScanTask task, DeleteFile... expectedDeleteFiles) } private CharSequenceSet deletePaths(FileScanTask task) { - return CharSequenceSet.of(Iterables.transform(task.deletes(), ContentFile::path)); + return CharSequenceSet.of(Iterables.transform(task.deletes(), ContentFile::location)); } private List planTasks() throws IOException { diff --git a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java index a1c72ae3b1e0..b1688e6653f2 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java @@ -178,7 +178,7 @@ public void testInsertDuplicatedKey() throws IOException { // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - assertThat(readRecordsAsList(table.schema(), dataFile.path())) + assertThat(readRecordsAsList(table.schema(), dataFile.location())) .isEqualTo( ImmutableList.of( createRecord(1, "aaa"), @@ -192,13 +192,13 @@ public void testInsertDuplicatedKey() throws IOException { // Check records in the pos-delete file. Schema posDeleteSchema = DeleteSchemaUtil.pathPosSchema(); - assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.path())) + assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.location())) .isEqualTo( ImmutableList.of( - posRecord.copy("file_path", dataFile.path(), "pos", 0L), - posRecord.copy("file_path", dataFile.path(), "pos", 1L), - posRecord.copy("file_path", dataFile.path(), "pos", 2L), - posRecord.copy("file_path", dataFile.path(), "pos", 3L))); + posRecord.copy("file_path", dataFile.location(), "pos", 0L), + posRecord.copy("file_path", dataFile.location(), "pos", 1L), + posRecord.copy("file_path", dataFile.location(), "pos", 2L), + posRecord.copy("file_path", dataFile.location(), "pos", 3L))); } @TestTemplate @@ -226,13 +226,13 @@ public void testUpsertSameRow() throws IOException { // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - assertThat(readRecordsAsList(table.schema(), dataFile.path())) + assertThat(readRecordsAsList(table.schema(), dataFile.location())) .isEqualTo(ImmutableList.of(record, record)); // Check records in the pos-delete file. DeleteFile posDeleteFile = result.deleteFiles()[0]; - assertThat(readRecordsAsList(DeleteSchemaUtil.pathPosSchema(), posDeleteFile.path())) - .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); + assertThat(readRecordsAsList(DeleteSchemaUtil.pathPosSchema(), posDeleteFile.location())) + .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.location(), "pos", 0L))); deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); @@ -312,7 +312,7 @@ public void testUpsertData() throws IOException { // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - assertThat(readRecordsAsList(table.schema(), dataFile.path())) + assertThat(readRecordsAsList(table.schema(), dataFile.location())) .isEqualTo( ImmutableList.of( createRecord(5, "aaa"), createRecord(6, "aaa"), createRecord(7, "ccc"))); @@ -320,7 +320,7 @@ public void testUpsertData() throws IOException { // Check records in the eq-delete file. DeleteFile eqDeleteFile = result.deleteFiles()[0]; assertThat(eqDeleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); - assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.path())) + assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.location())) .isEqualTo( ImmutableList.of(keyFunc.apply("aaa"), keyFunc.apply("ccc"), keyFunc.apply("bbb"))); @@ -328,8 +328,8 @@ public void testUpsertData() throws IOException { DeleteFile posDeleteFile = result.deleteFiles()[1]; Schema posDeleteSchema = DeleteSchemaUtil.pathPosSchema(); assertThat(posDeleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); - assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.path())) - .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); + assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.location())) + .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.location(), "pos", 0L))); } @TestTemplate @@ -397,7 +397,7 @@ public void testUpsertDataWithFullRowSchema() throws IOException { // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - assertThat(readRecordsAsList(table.schema(), dataFile.path())) + assertThat(readRecordsAsList(table.schema(), dataFile.location())) .isEqualTo( ImmutableList.of( createRecord(5, "aaa"), createRecord(6, "aaa"), createRecord(7, "ccc"))); @@ -405,7 +405,7 @@ public void testUpsertDataWithFullRowSchema() throws IOException { // Check records in the eq-delete file. DeleteFile eqDeleteFile = result.deleteFiles()[0]; assertThat(eqDeleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); - assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.path())) + assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.location())) .isEqualTo( ImmutableList.of( createRecord(3, "aaa"), createRecord(4, "ccc"), createRecord(2, "bbb"))); @@ -414,8 +414,8 @@ public void testUpsertDataWithFullRowSchema() throws IOException { DeleteFile posDeleteFile = result.deleteFiles()[1]; Schema posDeleteSchema = DeleteSchemaUtil.pathPosSchema(); assertThat(posDeleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); - assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.path())) - .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); + assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.location())) + .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.location(), "pos", 0L))); } @TestTemplate diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index 9c75a5e0f0fc..bf6f72cc287a 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -220,7 +220,7 @@ private static class FlinkDeleteFilter extends DeleteFilter { Schema tableSchema, Schema requestedSchema, InputFilesDecryptor inputFilesDecryptor) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + super(task.file().location(), task.deletes(), tableSchema, requestedSchema); this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); this.inputFilesDecryptor = inputFilesDecryptor; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index 344f64833b62..b6d6f60ef673 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -107,7 +107,7 @@ private String toString(Collection files) { .map( fileScanTask -> MoreObjects.toStringHelper(fileScanTask) - .add("file", fileScanTask.file().path().toString()) + .add("file", fileScanTask.file().location()) .add("start", fileScanTask.start()) .add("length", fileScanTask.length()) .toString()) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index ba0ea867ffb7..243c50a72b20 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -184,7 +184,7 @@ private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) for (FileScanTask scanTask : fileScanTasks) { long recordCountInFile = scanTask.file().recordCount(); - String[] splitFilePath = scanTask.file().path().toString().split("/"); + String[] splitFilePath = scanTask.file().location().split("/"); // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet // Writer ID: .......^^^^^ String filename = splitFilePath[splitFilePath.length - 1]; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index b283b8390a2b..44b1c57eda64 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -241,6 +241,6 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 8bfd6cb3d043..a01926783d05 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -114,7 +114,7 @@ public void testCloseTwice() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } } } @@ -133,7 +133,7 @@ public void testAbort() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); + assertThat(fs.exists(new Path(dataFile.location()))).isFalse(); } } } @@ -155,7 +155,7 @@ public void testCompleteFiles() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 5767fa02c822..f9c678f24089 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -161,10 +161,9 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -244,10 +243,10 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -316,11 +315,10 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover files appended in both snapshot1 and snapshot2 - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -406,10 +404,10 @@ public void testIncrementalFromSnapshotId() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -489,10 +487,10 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -529,12 +527,12 @@ public void testMaxPlanningSnapshotCount() throws Exception { ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); // should discover dataFile1 appended in snapshot1 verifyMaxPlanningSnapshotCountResult( - secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); + secondResult, null, snapshot1, ImmutableSet.of(dataFile1.location())); ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); // should discover dataFile2 appended in snapshot2 verifyMaxPlanningSnapshotCountResult( - thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); + thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.location())); } @Test @@ -670,7 +668,7 @@ private void verifyMaxPlanningSnapshotCountResult( assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index 9c75a5e0f0fc..bf6f72cc287a 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -220,7 +220,7 @@ private static class FlinkDeleteFilter extends DeleteFilter { Schema tableSchema, Schema requestedSchema, InputFilesDecryptor inputFilesDecryptor) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + super(task.file().location(), task.deletes(), tableSchema, requestedSchema); this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); this.inputFilesDecryptor = inputFilesDecryptor; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index 344f64833b62..b6d6f60ef673 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -107,7 +107,7 @@ private String toString(Collection files) { .map( fileScanTask -> MoreObjects.toStringHelper(fileScanTask) - .add("file", fileScanTask.file().path().toString()) + .add("file", fileScanTask.file().location()) .add("start", fileScanTask.start()) .add("length", fileScanTask.length()) .toString()) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 0b9c2193b4d5..04d7b8da6b9c 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -646,11 +646,11 @@ public void testSetCurrentAndCherryPickSnapshotId() { private void validateTableFiles(Table tbl, DataFile... expectedFiles) { tbl.refresh(); Set expectedFilePaths = - Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); + Arrays.stream(expectedFiles).map(DataFile::location).collect(Collectors.toSet()); Set actualFilePaths = StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) .map(FileScanTask::file) - .map(ContentFile::path) + .map(ContentFile::location) .collect(Collectors.toSet()); assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 8cebf950c5f0..3cab89e1ac7d 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -602,7 +602,7 @@ public static void assertEquals(ContentFile expected, ContentFile actual) assertThat(actual).isNotNull(); assertThat(actual.specId()).as("SpecId").isEqualTo(expected.specId()); assertThat(actual.content()).as("Content").isEqualTo(expected.content()); - assertThat(actual.path()).as("Path").isEqualTo(expected.path()); + assertThat(actual.location()).as("Location").isEqualTo(expected.location()); assertThat(actual.format()).as("Format").isEqualTo(expected.format()); assertThat(actual.partition().size()) .as("Partition size") diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java index 6b8399f666d4..aedc6edd6991 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -383,7 +383,7 @@ public void testRewriteAvoidRepeateCompress() throws IOException { assertThat(dataFilesRewrote).hasSize(2); // the biggest file do not be rewrote List rewroteDataFileNames = - dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); + dataFilesRewrote.stream().map(ContentFile::location).collect(Collectors.toList()); assertThat(rewroteDataFileNames).contains(file.getAbsolutePath()); // Assert the table records as expected. diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index ba0ea867ffb7..243c50a72b20 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -184,7 +184,7 @@ private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) for (FileScanTask scanTask : fileScanTasks) { long recordCountInFile = scanTask.file().recordCount(); - String[] splitFilePath = scanTask.file().path().toString().split("/"); + String[] splitFilePath = scanTask.file().location().split("/"); // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet // Writer ID: .......^^^^^ String filename = splitFilePath[splitFilePath.length - 1]; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index b283b8390a2b..44b1c57eda64 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -241,6 +241,6 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index 007d5f314946..69f39fba5130 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -224,7 +224,7 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } protected void testChangeLogs( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 8bfd6cb3d043..a01926783d05 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -114,7 +114,7 @@ public void testCloseTwice() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } } } @@ -133,7 +133,7 @@ public void testAbort() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); + assertThat(fs.exists(new Path(dataFile.location()))).isFalse(); } } } @@ -155,7 +155,7 @@ public void testCompleteFiles() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 5767fa02c822..f66b9e302924 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -105,7 +105,8 @@ private CycleResult verifyOneCycle( .hasSize(1) .first() .satisfies( - fileScanTask -> assertThat(fileScanTask.file().path()).isEqualTo(dataFile.path())); + fileScanTask -> + assertThat(fileScanTask.file().location()).isEqualTo(dataFile.location())); return new CycleResult(result.toPosition(), split); } @@ -161,10 +162,9 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -244,10 +244,10 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -316,11 +316,10 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover files appended in both snapshot1 and snapshot2 - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -406,10 +405,10 @@ public void testIncrementalFromSnapshotId() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -489,10 +488,10 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -529,12 +528,12 @@ public void testMaxPlanningSnapshotCount() throws Exception { ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); // should discover dataFile1 appended in snapshot1 verifyMaxPlanningSnapshotCountResult( - secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); + secondResult, null, snapshot1, ImmutableSet.of(dataFile1.location())); ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); // should discover dataFile2 appended in snapshot2 verifyMaxPlanningSnapshotCountResult( - thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); + thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.location())); } @Test @@ -670,7 +669,7 @@ private void verifyMaxPlanningSnapshotCountResult( assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java index 12bacdcd074d..4a21f451e1e5 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -170,7 +170,7 @@ private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit a for (int i = 0; i < expectedTasks.size(); ++i) { FileScanTask expectedTask = expectedTasks.get(i); FileScanTask actualTask = actualTasks.get(i); - assertThat(actualTask.file().path()).isEqualTo(expectedTask.file().path()); + assertThat(actualTask.file().location()).isEqualTo(expectedTask.file().location()); assertThat(actualTask.sizeBytes()).isEqualTo(expectedTask.sizeBytes()); assertThat(actualTask.filesCount()).isEqualTo(expectedTask.filesCount()); assertThat(actualTask.start()).isEqualTo(expectedTask.start()); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index 9c75a5e0f0fc..bf6f72cc287a 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -220,7 +220,7 @@ private static class FlinkDeleteFilter extends DeleteFilter { Schema tableSchema, Schema requestedSchema, InputFilesDecryptor inputFilesDecryptor) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + super(task.file().location(), task.deletes(), tableSchema, requestedSchema); this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); this.inputFilesDecryptor = inputFilesDecryptor; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index 344f64833b62..b6d6f60ef673 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -107,7 +107,7 @@ private String toString(Collection files) { .map( fileScanTask -> MoreObjects.toStringHelper(fileScanTask) - .add("file", fileScanTask.file().path().toString()) + .add("file", fileScanTask.file().location()) .add("start", fileScanTask.start()) .add("length", fileScanTask.length()) .toString()) diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 0b9c2193b4d5..04d7b8da6b9c 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -646,11 +646,11 @@ public void testSetCurrentAndCherryPickSnapshotId() { private void validateTableFiles(Table tbl, DataFile... expectedFiles) { tbl.refresh(); Set expectedFilePaths = - Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); + Arrays.stream(expectedFiles).map(DataFile::location).collect(Collectors.toSet()); Set actualFilePaths = StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) .map(FileScanTask::file) - .map(ContentFile::path) + .map(ContentFile::location) .collect(Collectors.toSet()); assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 8cebf950c5f0..3cab89e1ac7d 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -602,7 +602,7 @@ public static void assertEquals(ContentFile expected, ContentFile actual) assertThat(actual).isNotNull(); assertThat(actual.specId()).as("SpecId").isEqualTo(expected.specId()); assertThat(actual.content()).as("Content").isEqualTo(expected.content()); - assertThat(actual.path()).as("Path").isEqualTo(expected.path()); + assertThat(actual.location()).as("Location").isEqualTo(expected.location()); assertThat(actual.format()).as("Format").isEqualTo(expected.format()); assertThat(actual.partition().size()) .as("Partition size") diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java index 085e7e48204c..a4dde8af3c12 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -390,7 +390,7 @@ public void testRewriteAvoidRepeateCompress() throws IOException { assertThat(dataFilesRewrote).hasSize(2); // the biggest file do not be rewrote List rewroteDataFileNames = - dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); + dataFilesRewrote.stream().map(ContentFile::location).collect(Collectors.toList()); assertThat(rewroteDataFileNames).contains(file.getAbsolutePath()); // Assert the table records as expected. diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index ba0ea867ffb7..243c50a72b20 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -184,7 +184,7 @@ private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) for (FileScanTask scanTask : fileScanTasks) { long recordCountInFile = scanTask.file().recordCount(); - String[] splitFilePath = scanTask.file().path().toString().split("/"); + String[] splitFilePath = scanTask.file().location().split("/"); // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet // Writer ID: .......^^^^^ String filename = splitFilePath[splitFilePath.length - 1]; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index b283b8390a2b..44b1c57eda64 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -241,6 +241,6 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index 007d5f314946..69f39fba5130 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -224,7 +224,7 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } protected void testChangeLogs( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 8bfd6cb3d043..a01926783d05 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -114,7 +114,7 @@ public void testCloseTwice() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } } } @@ -133,7 +133,7 @@ public void testAbort() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); + assertThat(fs.exists(new Path(dataFile.location()))).isFalse(); } } } @@ -155,7 +155,7 @@ public void testCompleteFiles() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 5767fa02c822..f66b9e302924 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -105,7 +105,8 @@ private CycleResult verifyOneCycle( .hasSize(1) .first() .satisfies( - fileScanTask -> assertThat(fileScanTask.file().path()).isEqualTo(dataFile.path())); + fileScanTask -> + assertThat(fileScanTask.file().location()).isEqualTo(dataFile.location())); return new CycleResult(result.toPosition(), split); } @@ -161,10 +162,9 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -244,10 +244,10 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -316,11 +316,10 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover files appended in both snapshot1 and snapshot2 - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -406,10 +405,10 @@ public void testIncrementalFromSnapshotId() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -489,10 +488,10 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -529,12 +528,12 @@ public void testMaxPlanningSnapshotCount() throws Exception { ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); // should discover dataFile1 appended in snapshot1 verifyMaxPlanningSnapshotCountResult( - secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); + secondResult, null, snapshot1, ImmutableSet.of(dataFile1.location())); ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); // should discover dataFile2 appended in snapshot2 verifyMaxPlanningSnapshotCountResult( - thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); + thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.location())); } @Test @@ -670,7 +669,7 @@ private void verifyMaxPlanningSnapshotCountResult( assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java index 12bacdcd074d..4a21f451e1e5 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -170,7 +170,7 @@ private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit a for (int i = 0; i < expectedTasks.size(); ++i) { FileScanTask expectedTask = expectedTasks.get(i); FileScanTask actualTask = actualTasks.get(i); - assertThat(actualTask.file().path()).isEqualTo(expectedTask.file().path()); + assertThat(actualTask.file().location()).isEqualTo(expectedTask.file().location()); assertThat(actualTask.sizeBytes()).isEqualTo(expectedTask.sizeBytes()); assertThat(actualTask.filesCount()).isEqualTo(expectedTask.filesCount()); assertThat(actualTask.start()).isEqualTo(expectedTask.start()); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java index 13c459128dec..5650c4e82458 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java @@ -497,7 +497,7 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio String file1Location = appendData(table, "file1"); List tasks = Lists.newArrayList(table.newScan().planFiles()); assertThat(tasks).as("Should scan 1 file").hasSize(1); - assertThat(file1Location).isEqualTo(tasks.get(0).file().path()); + assertThat(file1Location).isEqualTo(tasks.get(0).file().location()); // collect metadata file List metadataFiles = @@ -528,7 +528,7 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio tasks = Lists.newArrayList(table.newScan().planFiles()); assertThat(tasks).as("Should scan 2 files").hasSize(2); Set files = - tasks.stream().map(task -> task.file().path().toString()).collect(Collectors.toSet()); + tasks.stream().map(task -> task.file().location()).collect(Collectors.toSet()); assertThat(files).contains(file1Location, file2Location); } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java index b09540fdc9f6..845892b3d817 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -33,6 +33,7 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.RowDelta; @@ -209,7 +210,7 @@ private void commitToTable( .filter(payload -> payload.dataFiles() != null) .flatMap(payload -> payload.dataFiles().stream()) .filter(dataFile -> dataFile.recordCount() > 0) - .filter(distinctByKey(dataFile -> dataFile.path().toString())) + .filter(distinctByKey(ContentFile::location)) .collect(Collectors.toList()); List deleteFiles = @@ -217,7 +218,7 @@ private void commitToTable( .filter(payload -> payload.deleteFiles() != null) .flatMap(payload -> payload.deleteFiles().stream()) .filter(deleteFile -> deleteFile.recordCount() > 0) - .filter(distinctByKey(deleteFile -> deleteFile.path().toString())) + .filter(distinctByKey(ContentFile::location)) .collect(Collectors.toList()); if (terminated) { diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java index 8b9cd3d3d4db..5b1b8222534f 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java @@ -277,8 +277,10 @@ public void abortJob(JobContext originalContext, int status) throws IOException .onFailure( (file, exc) -> LOG.warn( - "Failed to remove data file {} on abort job", file.path(), exc)) - .run(file -> table.io().deleteFile(file.path().toString())); + "Failed to remove data file {} on abort job", + file.location(), + exc)) + .run(file -> table.io().deleteFile(file.location())); } }); } finally { diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java index 0c698aa4b2fd..9fc578e88a62 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java @@ -113,7 +113,7 @@ public void close(boolean abort) throws IOException { .onFailure( (file, exception) -> LOG.debug("Failed on to remove file {} on abort", file, exception)) - .run(dataFile -> io.deleteFile(dataFile.path().toString())); + .run(dataFile -> io.deleteFile(dataFile.location())); } LOG.info( diff --git a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index 9b8d4e9247a2..492729d97338 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -320,8 +320,7 @@ private CloseableIterable openTask(FileScanTask currentTask, Schema readSchem DataFile file = currentTask.file(); InputFile inputFile = encryptionManager.decrypt( - EncryptedFiles.encryptedInput( - io.newInputFile(file.path().toString()), file.keyMetadata())); + EncryptedFiles.encryptedInput(io.newInputFile(file.location()), file.keyMetadata())); CloseableIterable iterable; switch (file.format()) { @@ -336,7 +335,7 @@ private CloseableIterable openTask(FileScanTask currentTask, Schema readSchem break; default: throw new UnsupportedOperationException( - String.format("Cannot read %s file: %s", file.format().name(), file.path())); + String.format("Cannot read %s file: %s", file.format().name(), file.location())); } return iterable; diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java index e026c0e4c115..a8a8c3f570e8 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java @@ -85,7 +85,7 @@ private List stripeOffsetsFromReader(DataFile dataFile) throws IOException private List stripeOffsetsFromReader(DataFile dataFile, OrcFile.ReaderOptions options) throws IOException { - return OrcFile.createReader(new Path(dataFile.path().toString()), options).getStripes().stream() + return OrcFile.createReader(new Path(dataFile.location()), options).getStripes().stream() .map(StripeInformation::getOffset) .collect(Collectors.toList()); } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 9c3c7e589c07..17820b14b011 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -300,7 +300,7 @@ public void testExpireDeleteFiles() throws Exception { Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); - Path deleteFilePath = new Path(String.valueOf(deleteFile.path())); + Path deleteFilePath = new Path(deleteFile.location()); sql( "CALL %s.system.rewrite_data_files(" diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index e1966520893a..76ef5cfc9695 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -433,8 +433,7 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); - Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); + Path deleteFilePath = new Path(TestHelpers.deleteFiles(table).iterator().next().location()); // wait to ensure files are old enough waitUntilAfter(System.currentTimeMillis()); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java index da9b61f4d74d..7175ad4ec499 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -305,7 +305,7 @@ private void writePosDeletesForFiles(Table table, List files) throws I List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < DELETE_FILE_SIZE; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java index 63af42ff4c4f..4dc2e852aae9 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -1131,7 +1131,7 @@ public void testUpdateWithStaticPredicatePushdown() { // remove the data file from the 'hr' partition to ensure it is not scanned DataFile dataFile = Iterables.getOnlyElement(snapshot.addedDataFiles(table.io())); - table.io().deleteFile(dataFile.path().toString()); + table.io().deleteFile(dataFile.location()); // disable dynamic pruning and rely only on static predicate pushdown withSQLConf( diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 2ac3de2ff947..60430339d392 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -45,7 +45,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, 0.25, numDeleteFile); + writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } } diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index 8cd6fb36fcf5..72fffaa50b8e 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -47,7 +47,7 @@ protected void appendData() throws IOException { // add pos-deletes table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, percentDeleteRow); + writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } } diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index 1ae48e213cb7..892e2c5db19f 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -48,7 +48,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { writePosDeletesWithNoise( - file.path(), + file.location(), NUM_ROWS, PERCENT_DELETE_ROW, (int) (percentUnrelatedDeletes / PERCENT_DELETE_ROW), diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index dfd0b58ffbee..7ab17361b13e 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; @@ -672,7 +673,7 @@ private static void importUnpartitionedSparkTable( if (checkDuplicateFiles) { Dataset importedFiles = spark - .createDataset(Lists.transform(files, f -> f.path().toString()), Encoders.STRING()) + .createDataset(Lists.transform(files, ContentFile::location), Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -800,7 +801,7 @@ public static void importSparkPartitions( if (checkDuplicateFiles) { Dataset importedFiles = filesToImport - .map((MapFunction) f -> f.path().toString(), Encoders.STRING()) + .map((MapFunction) ContentFile::location, Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -819,7 +820,7 @@ public static void importSparkPartitions( .repartition(numShufflePartitions) .map( (MapFunction>) - file -> Tuple2.apply(file.path().toString(), file), + file -> Tuple2.apply(file.location(), file), Encoders.tuple(Encoders.STRING(), Encoders.javaSerialization(DataFile.class))) .orderBy(col("_1")) .mapPartitions( diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index c5b80835833e..e98494484b69 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -440,7 +440,7 @@ public CloseableIterator entries(ManifestFileBean manifest) { } static FileInfo toFileInfo(ContentFile file) { - return new FileInfo(file.path().toString(), file.content().toString()); + return new FileInfo(file.location(), file.content().toString()); } } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index 96ff430c179d..40d907e12c08 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -148,7 +148,7 @@ public boolean next() throws IOException { if (currentTask != null && !currentTask.isDataTask()) { String filePaths = referencedFiles(currentTask) - .map(file -> file.path().toString()) + .map(ContentFile::location) .collect(Collectors.joining(", ")); LOG.error("Error reading file(s): {}", filePaths, e); } @@ -194,7 +194,7 @@ private Map inputFiles() { } private EncryptedInputFile toEncryptedInputFile(ContentFile file) { - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); return EncryptedFiles.encryptedInput(inputFile, file.keyMetadata()); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index a2cb74c926c9..f45c152203ee 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -82,7 +82,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); // update the current file for Spark's filename() function diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index 25cd9eda6bce..a6e2891ae696 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -110,13 +110,13 @@ private CloseableIterable openChangelogScanTask(ChangelogScanTask t } CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); @@ -125,7 +125,7 @@ private CloseableIterable openDeletedDataFileScanTask(DeletedDataFi private CloseableIterable rows(ContentScanTask task, Schema readSchema) { Map idToConstant = constantsMap(task, readSchema); - String filePath = task.file().path().toString(); + String filePath = task.file().location(); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java index 7aa5a971562a..ee9449ee13c8 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java @@ -41,7 +41,7 @@ public EqualityDeleteRowReader( @Override protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter matches = - new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter(), true); + new SparkDeleteFilter(task.file().location(), task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = matches.requiredSchema(); @@ -49,7 +49,7 @@ protected CloseableIterator open(FileScanTask task) { DataFile file = task.file(); // update the current file for Spark's filename() function - InputFileBlockHolder.set(file.path().toString(), task.start(), task.length()); + InputFileBlockHolder.set(file.location(), task.start(), task.length()); return matches.findEqualityDeleteRows(open(task, requiredSchema, idToConstant)).iterator(); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java index 4b847474153c..1a894df29166 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java @@ -74,13 +74,13 @@ protected Stream> referencedFiles(PositionDeletesScanTask task) { @Override protected CloseableIterator open(PositionDeletesScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening position delete file {}", filePath); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with %s", task); // select out constant fields when pushing down filter to row reader diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 33b1d6275d6e..f24602fd5583 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -81,7 +81,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); SparkDeleteFilter deleteFilter = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); @@ -101,7 +101,7 @@ protected CloseableIterable open( if (task.isDataTask()) { return newDataIterable(task.asDataTask(), readSchema); } else { - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull( inputFile, "Could not find InputFile associated with FileScanTask"); return newIterable( diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java index a103a5003222..5f343128161d 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java @@ -81,7 +81,7 @@ private static String taskInfo() { * @param files a list of files to delete */ public static void deleteFiles(String context, FileIO io, List> files) { - List paths = Lists.transform(files, file -> file.path().toString()); + List paths = Lists.transform(files, ContentFile::location); deletePaths(context, io, paths); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index d978b81e67bd..345a3bb75797 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -126,7 +126,7 @@ public void filter(Filter[] filters) { this.filteredLocations = fileLocations; List filteredTasks = tasks().stream() - .filter(file -> fileLocations.contains(file.file().path().toString())) + .filter(file -> fileLocations.contains(file.file().location())) .collect(Collectors.toList()); LOG.info( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java index c44bacf149b5..b241b8ed6363 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java @@ -60,7 +60,8 @@ public static void assertEquals(FileScanTask expected, FileScanTask actual) { } public static void assertEquals(DataFile expected, DataFile actual) { - Assert.assertEquals("Should match the serialized record path", expected.path(), actual.path()); + Assert.assertEquals( + "Should match the serialized record path", expected.location(), actual.location()); Assert.assertEquals( "Should match the serialized record format", expected.format(), actual.format()); Assert.assertEquals( @@ -105,7 +106,7 @@ private static List getFileScanTasksInFilePathOrder( ScanTaskGroup taskGroup) { return taskGroup.tasks().stream() // use file path + start position to differentiate the tasks - .sorted(Comparator.comparing(o -> o.file().path().toString() + "##" + o.start())) + .sorted(Comparator.comparing(o -> o.file().location() + "##" + o.start())) .collect(Collectors.toList()); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java index b0b3085dca70..dac09fea7562 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java @@ -42,7 +42,7 @@ public static List snapshotIds(Long... ids) { } public static List files(ContentFile... files) { - return Arrays.stream(files).map(file -> file.path().toString()).collect(Collectors.toList()); + return Arrays.stream(files).map(ContentFile::location).collect(Collectors.toList()); } public static void validateDataManifest( @@ -62,7 +62,7 @@ public static void validateDataManifest( actualDataSeqs.add(entry.dataSequenceNumber()); actualFileSeqs.add(entry.fileSequenceNumber()); actualSnapshotIds.add(entry.snapshotId()); - actualFiles.add(entry.file().path().toString()); + actualFiles.add(entry.file().location()); } assertSameElements("data seqs", actualDataSeqs, dataSeqs); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java index 82b36528996f..3e8953fb950c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java @@ -142,7 +142,7 @@ protected static String dbPath(String dbName) { } protected void withUnavailableFiles(Iterable> files, Action action) { - Iterable fileLocations = Iterables.transform(files, file -> file.path().toString()); + Iterable fileLocations = Iterables.transform(files, ContentFile::location); withUnavailableLocations(fileLocations, action); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 6d1bbc2c18f1..65a21e28a712 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -204,7 +204,7 @@ public void dataFilesCleanupWithParallelTasks() { .forEach( file -> Assert.assertTrue( - "FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()))); + "FILE_A should be deleted", deletedFiles.contains(FILE_A.location()))); checkRemoveFilesResults(4L, 0, 0, 6L, 4L, 6, result); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 71339b32421a..72294fdbdd76 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -239,8 +239,8 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { Sets.newHashSet( "remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3")); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); - Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); + Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.location())); checkExpirationResults(2L, 0L, 0L, 3L, 3L, result); } @@ -555,7 +555,7 @@ public void testScanExpiredManifestInValidSnapshotAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -584,7 +584,7 @@ public void testScanExpiredManifestInValidSnapshotFastAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -627,7 +627,7 @@ public void testWithExpiringDanglingStageCommit() { .addedDataFiles(table.io()) .forEach( i -> { - expectedDeletes.add(i.path().toString()); + expectedDeletes.add(i.location()); }); // ManifestList should be deleted too @@ -696,7 +696,7 @@ public void testWithCherryPickTableSnapshot() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); @@ -745,7 +745,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); checkExpirationResults(0L, 0L, 0L, 1L, 1L, firstResult); @@ -765,7 +765,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); checkExpirationResults(0L, 0L, 0L, 0L, 2L, secondResult); @@ -866,7 +866,7 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted + FILE_A.location()), // deleted deletedFiles); checkExpirationResults(1, 0, 0, 2, 2, result); @@ -935,7 +935,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted + FILE_A.location()), // deleted deletedFiles); checkExpirationResults(1, 0, 0, 1, 2, result); @@ -1049,7 +1049,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back + FILE_B.location()), // added, but rolled back deletedFiles); checkExpirationResults(1, 0, 0, 1, 1, result); @@ -1098,9 +1098,9 @@ public void testExpireOlderThanWithDeleteFile() { secondSnapshot.manifestListLocation(), thirdSnapshot.manifestListLocation(), fourthSnapshot.manifestListLocation(), - FILE_A.path().toString(), - FILE_A_POS_DELETES.path().toString(), - FILE_A_EQ_DELETES.path().toString()); + FILE_A.location(), + FILE_A_POS_DELETES.location(), + FILE_A_EQ_DELETES.location()); expectedDeletes.addAll( thirdSnapshot.allManifests(table.io()).stream() @@ -1273,7 +1273,7 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe .withPartitionPath("c1=1") .withRecordCount(1) .build(); - dataFiles.add(df.path().toString()); + dataFiles.add(df.location()); table.newFastAppend().appendFile(df).commit(); } @@ -1346,9 +1346,9 @@ public void testExpireSomeCheckFilesDeleted() { // C, D should be retained (live) // B should be retained (previous snapshot points to it) // A should be deleted - Assert.assertTrue(deletedFiles.contains(FILE_A.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_B.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_C.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_D.path().toString())); + Assert.assertTrue(deletedFiles.contains(FILE_A.location())); + Assert.assertFalse(deletedFiles.contains(FILE_B.location())); + Assert.assertFalse(deletedFiles.contains(FILE_C.location())); + Assert.assertFalse(deletedFiles.contains(FILE_D.location())); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index c978be9e3d91..6b7d5b9e1cfd 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1814,8 +1814,7 @@ private void writeDF(Dataset df) { private List writePosDeletesToFile( Table table, DataFile dataFile, int outputDeleteFiles) { - return writePosDeletes( - table, dataFile.partition(), dataFile.path().toString(), outputDeleteFiles); + return writePosDeletes(table, dataFile.partition(), dataFile.location(), outputDeleteFiles); } private List writePosDeletes( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 43b762813de2..15d4b45ab902 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -831,7 +831,7 @@ private void writePosDeletesForFiles( List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < deletesPerDataFile; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables @@ -868,17 +868,17 @@ private List deleteFiles(Table table) { private > List except(List first, List second) { Set secondPaths = - second.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + second.stream().map(ContentFile::location).collect(Collectors.toSet()); return first.stream() - .filter(f -> !secondPaths.contains(f.path().toString())) + .filter(f -> !secondPaths.contains(f.location())) .collect(Collectors.toList()); } private void assertNotContains(List original, List rewritten) { Set originalPaths = - original.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + original.stream().map(ContentFile::location).collect(Collectors.toSet()); Set rewrittenPaths = - rewritten.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + rewritten.stream().map(ContentFile::location).collect(Collectors.toSet()); rewrittenPaths.retainAll(originalPaths); Assert.assertEquals(0, rewrittenPaths.size()); } @@ -887,7 +887,7 @@ private void assertLocallySorted(List deleteFiles) { for (DeleteFile deleteFile : deleteFiles) { Dataset deletes = spark.read().format("iceberg").load("default." + TABLE_NAME + ".position_deletes"); - deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.path().toString())); + deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.location())); List rows = deletes.collectAsList(); Assert.assertFalse("Empty delete file found", rows.isEmpty()); int lastPos = 0; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java index e5e0e350e50b..d417acdeb20a 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java @@ -114,7 +114,7 @@ public Boolean hasIterator(FileScanTask task) { } private String getKey(FileScanTask task) { - return task.file().path().toString(); + return task.file().location(); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 693bcec9eb05..b5c055925bf7 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -197,8 +197,8 @@ private void writeAndValidateWithLocations(Table table, File location, File expe Assert.assertTrue( String.format( "File should have the parent directory %s, but has: %s.", - expectedDataDir.getAbsolutePath(), dataFile.path()), - URI.create(dataFile.path().toString()) + expectedDataDir.getAbsolutePath(), dataFile.location()), + URI.create(dataFile.location()) .getPath() .startsWith(expectedDataDir.getAbsolutePath()))); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 9cee25560366..01424a1abdfe 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -109,7 +109,7 @@ public void testWriteFormatOptionOverridesTableProperties() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); Assert.assertEquals(FileFormat.PARQUET, fileFormat); }); } @@ -134,7 +134,7 @@ public void testNoWriteFormatOption() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); Assert.assertEquals(FileFormat.AVRO, fileFormat); }); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index d82bc0b4ba8e..99c621997701 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -268,7 +268,7 @@ public void testEntriesTableDataFilePrune() throws Exception { .select("data_file.file_path") .collectAsList()); - List singleExpected = ImmutableList.of(row(file.path())); + List singleExpected = ImmutableList.of(row(file.location())); assertEquals( "Should prune a single element from a nested struct", singleExpected, singleActual); @@ -307,7 +307,7 @@ public void testEntriesTableDataFilePruneMulti() throws Exception { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a nested struct", multiExpected, multiActual); } @@ -341,7 +341,7 @@ public void testFilesSelectMap() throws Exception { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a row", multiExpected, multiActual); } @@ -2316,7 +2316,7 @@ private DeleteFile writePosDeleteFile(Table table, long pos) { StructLike dataFilePartition = dataFile.partition(); PositionDelete delete = PositionDelete.create(); - delete.set(dataFile.path(), pos, null); + delete.set(dataFile.location(), pos, null); return writePositionDeletes(table, dataFileSpec, dataFilePartition, ImmutableList.of(delete)); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java index 2ec4f2f4f907..af3ec224cb4f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -26,6 +26,7 @@ import java.util.UUID; import java.util.function.BiFunction; import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -140,8 +141,8 @@ public void testNullRows() throws IOException { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); @@ -150,9 +151,9 @@ public void testNullRows() throws IOException { StructLikeSet actual = actual(tableName, tab); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); - StructLikeSet expected = - expected(tab, expectedDeletes, null, posDeletes.first().path().toString()); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); + StructLikeSet expected = expected(tab, expectedDeletes, null, posDeletes.first().location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -181,7 +182,7 @@ public void testPartitionedTable() throws IOException { GenericRecord partitionB = GenericRecord.create(tab.spec().partitionType()); partitionB.setField("data", "b"); StructLikeSet expected = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -221,7 +222,7 @@ public void testSelect() throws IOException { (delete, file) -> { int rowData = delete.get(2, GenericRecord.class).get(0, Integer.class); long pos = delete.get(1, Long.class); - return row(rowData, pos, file.path().toString(), file.path().toString()); + return row(rowData, pos, file.location(), file.location()); }; expected.addAll( deletesA.first().stream() @@ -270,7 +271,7 @@ public void testSplitTasks() throws IOException { List> deletes = Lists.newArrayList(); for (long i = 0; i < records; i++) { - deletes.add(positionDelete(tab.schema(), dFile.path(), i, (int) i, String.valueOf(i))); + deletes.add(positionDelete(tab.schema(), dFile.location(), i, (int) i, String.valueOf(i))); } DeleteFile posDeletes = FileHelpers.writePosDeleteFile( @@ -292,7 +293,7 @@ public void testSplitTasks() throws IOException { } StructLikeSet actual = actual(tableName, tab); - StructLikeSet expected = expected(tab, deletes, null, posDeletes.path().toString()); + StructLikeSet expected = expected(tab, deletes, null, posDeletes.location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -322,9 +323,9 @@ public void testPartitionFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data", "a"); Record partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTab.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -365,9 +366,9 @@ public void testPartitionTransformFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data_trunc", "a"); Record partitionB = partitionRecordTemplate.copy("data_trunc", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTable.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -415,7 +416,7 @@ public void testPartitionEvolutionReplace() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -427,7 +428,7 @@ public void testPartitionEvolutionReplace() throws Exception { deletes10.first(), partition10, tab.spec().specId(), - deletes10.second().path().toString()); + deletes10.second().location()); StructLikeSet actual10 = actual(tableName, tab, "partition.id = 10 AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expected10, actual10); @@ -465,7 +466,7 @@ public void testPartitionEvolutionAdd() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -477,7 +478,7 @@ public void testPartitionEvolutionAdd() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId0, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -520,7 +521,7 @@ public void testPartitionEvolutionRemove() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -532,7 +533,7 @@ public void testPartitionEvolutionRemove() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId1, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -577,7 +578,7 @@ public void testSpecIdFilter() throws Exception { deletesUnpartitioned.first(), partitionRecordTemplate, unpartitionedSpec, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, String.format("spec_id = %d", unpartitionedSpec)); Assert.assertEquals( @@ -589,9 +590,9 @@ public void testSpecIdFilter() throws Exception { StructLike partitionA = partitionRecordTemplate.copy("data", "a"); StructLike partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expected = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); expected.addAll( - expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().path().toString())); + expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().location())); StructLikeSet actual = actual(tableName, tab, String.format("spec_id = %d", dataSpec)); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); @@ -645,14 +646,14 @@ public void testSchemaEvolutionAdd() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); @@ -707,14 +708,14 @@ public void testSchemaEvolutionRemove() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); @@ -792,8 +793,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); @@ -827,7 +828,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { actual(tableName, tab, null, ImmutableList.of("file_path", "pos", "row", "spec_id")); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); StructLikeSet expected = expected(tab, expectedDeletes, null, null); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); @@ -846,8 +848,8 @@ public void testWriteMixedRows() throws Exception { // Add a delete file with row and without row List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dataFileA.path(), 0L)); - deletes.add(Pair.of(dataFileA.path(), 1L)); + deletes.add(Pair.of(dataFileA.location(), 0L)); + deletes.add(Pair.of(dataFileA.location(), 1L)); Pair deletesWithoutRow = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of("a"), deletes); @@ -905,7 +907,7 @@ public void testWriteMixedRows() throws Exception { expected( tab, Lists.newArrayList( - positionDelete(dataFileA.path(), 0L), positionDelete(dataFileA.path(), 1L)), + positionDelete(dataFileA.location(), 0L), positionDelete(dataFileA.location(), 1L)), partitionA, null)); allExpected.addAll(expected(tab, deletesWithRow.first(), partitionB, null)); @@ -1481,13 +1483,13 @@ private Pair>, DeleteFile> deleteFile( Lists.newArrayList( positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 0L, idPartition != null ? idPartition : 29, dataPartition != null ? dataPartition : "c"), positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 1L, idPartition != null ? idPartition : 61, dataPartition != null ? dataPartition : "r")); @@ -1533,9 +1535,9 @@ private void commit( Assert.assertEquals(expectedTargetFiles, addedFiles.size()); List sortedAddedFiles = - addedFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + addedFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); List sortedRewrittenFiles = - rewrittenFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + rewrittenFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); Assert.assertNotEquals("Lists should not be the same", sortedAddedFiles, sortedRewrittenFiles); baseTab diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java index beaf7b75c6c0..d59c280bc060 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java @@ -444,7 +444,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set matchingFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().filter(filter).planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); matchingFileLocations.add(path); } } catch (IOException e) { @@ -454,7 +454,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set deletedFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); if (!matchingFileLocations.contains(path)) { io.deleteFile(path); deletedFileLocations.add(path); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index b1f2082b5d9b..e03ba3794a18 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -193,7 +193,7 @@ private void checkSparkDataFile(Table table) throws IOException { } private void checkDataFile(DataFile expected, DataFile actual) { - Assert.assertEquals("Path must match", expected.path(), actual.path()); + Assert.assertEquals("Path must match", expected.location(), actual.location()); Assert.assertEquals("Format must match", expected.format(), actual.format()); Assert.assertEquals("Record count must match", expected.recordCount(), actual.recordCount()); Assert.assertEquals("Size must match", expected.fileSizeInBytes(), actual.fileSizeInBytes()); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index bbdd49b63df8..dda49b49465c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -343,10 +343,10 @@ public void testPosDeletesAllRowsInBatch() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -376,10 +376,10 @@ public void testPosDeletesWithDeletedColumn() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -455,8 +455,8 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 3L), // id = 89 - Pair.of(dataFile.path(), 5L) // id = 121 + Pair.of(dataFile.location(), 3L), // id = 89 + Pair.of(dataFile.location(), 5L) // id = 121 ); Pair posDeletes = @@ -486,10 +486,10 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { public void testFilterOnDeletedMetadataColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -611,13 +611,13 @@ public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOExceptio // Add positional deletes to the table List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 97L), - Pair.of(dataFile.path(), 98L), - Pair.of(dataFile.path(), 99L), - Pair.of(dataFile.path(), 101L), - Pair.of(dataFile.path(), 103L), - Pair.of(dataFile.path(), 107L), - Pair.of(dataFile.path(), 109L)); + Pair.of(dataFile.location(), 97L), + Pair.of(dataFile.location(), 98L), + Pair.of(dataFile.location(), 99L), + Pair.of(dataFile.location(), 101L), + Pair.of(dataFile.location(), 103L), + Pair.of(dataFile.location(), 107L), + Pair.of(dataFile.location(), 109L)); Pair posDeletes = FileHelpers.writeDeleteFile( table, diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 8c355612ceb5..d130ccbc2cd1 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -292,7 +292,7 @@ public void testExpireDeleteFiles() throws Exception { Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); - Path deleteFilePath = new Path(String.valueOf(deleteFile.path())); + Path deleteFilePath = new Path(deleteFile.location()); sql( "CALL %s.system.rewrite_data_files(" diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 3b1ca1b239b8..01e341eead89 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -424,8 +424,7 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); - Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); + Path deleteFilePath = new Path(TestHelpers.deleteFiles(table).iterator().next().location()); // wait to ensure files are old enough waitUntilAfter(System.currentTimeMillis()); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java index 4c5082cabbb6..2d3d3b851a99 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -314,7 +314,7 @@ private void writePosDeletesForFiles(Table table, List files) throws I List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < DELETE_FILE_SIZE; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSparkExecutorCache.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSparkExecutorCache.java index 3d995cc4f096..ed0685735941 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSparkExecutorCache.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSparkExecutorCache.java @@ -222,7 +222,7 @@ private void checkMerge(RowLevelOperationMode mode) throws Exception { } private int streamCount(DeleteFile deleteFile) { - CustomInputFile inputFile = INPUT_FILES.get(deleteFile.path().toString()); + CustomInputFile inputFile = INPUT_FILES.get(deleteFile.location()); return inputFile.streamCount(); } @@ -247,7 +247,7 @@ private List createAndInitTable(String operation, RowLevelOperationM List> posDeletes = dataFiles(table).stream() - .map(dataFile -> Pair.of(dataFile.path(), 0L)) + .map(dataFile -> Pair.of((CharSequence) dataFile.location(), 0L)) .collect(Collectors.toList()); Pair posDeleteResult = writePosDeletes(table, posDeletes); DeleteFile posDeleteFile = posDeleteResult.first(); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java index e84789de7abc..9968f5eb0f2c 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -1299,7 +1299,7 @@ public void testUpdateWithStaticPredicatePushdown() { // remove the data file from the 'hr' partition to ensure it is not scanned DataFile dataFile = Iterables.getOnlyElement(snapshot.addedDataFiles(table.io())); - table.io().deleteFile(dataFile.path().toString()); + table.io().deleteFile(dataFile.location()); // disable dynamic pruning and rely only on static predicate pushdown withSQLConf( diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 2ac3de2ff947..60430339d392 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -45,7 +45,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, 0.25, numDeleteFile); + writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } } diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index 8cd6fb36fcf5..72fffaa50b8e 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -47,7 +47,7 @@ protected void appendData() throws IOException { // add pos-deletes table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, percentDeleteRow); + writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } } diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index 1ae48e213cb7..892e2c5db19f 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -48,7 +48,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { writePosDeletesWithNoise( - file.path(), + file.location(), NUM_ROWS, PERCENT_DELETE_ROW, (int) (percentUnrelatedDeletes / PERCENT_DELETE_ROW), diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index e103104171f6..aaeabe350f54 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; @@ -635,7 +636,7 @@ private static void importUnpartitionedSparkTable( if (checkDuplicateFiles) { Dataset importedFiles = spark - .createDataset(Lists.transform(files, f -> f.path().toString()), Encoders.STRING()) + .createDataset(Lists.transform(files, ContentFile::location), Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -763,7 +764,7 @@ public static void importSparkPartitions( if (checkDuplicateFiles) { Dataset importedFiles = filesToImport - .map((MapFunction) f -> f.path().toString(), Encoders.STRING()) + .map((MapFunction) ContentFile::location, Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -782,7 +783,7 @@ public static void importSparkPartitions( .repartition(numShufflePartitions) .map( (MapFunction>) - file -> Tuple2.apply(file.path().toString(), file), + file -> Tuple2.apply(file.location(), file), Encoders.tuple(Encoders.STRING(), Encoders.javaSerialization(DataFile.class))) .orderBy(col("_1")) .mapPartitions( diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index cff07c05d423..13ce67cda18e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -433,7 +433,7 @@ public CloseableIterator entries(ManifestFileBean manifest) { } static FileInfo toFileInfo(ContentFile file) { - return new FileInfo(file.path().toString(), file.content().toString()); + return new FileInfo(file.location(), file.content().toString()); } } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java index b9dc46f5e1bc..c8c883e553b7 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java @@ -88,7 +88,7 @@ Result doExecute() { RewriteFiles rewriteFiles = table.newRewrite(); List danglingDeletes = findDanglingDeletes(); for (DeleteFile deleteFile : danglingDeletes) { - LOG.debug("Removing dangling delete file {}", deleteFile.path()); + LOG.debug("Removing dangling delete file {}", deleteFile.location()); rewriteFiles.deleteFile(deleteFile); } if (!danglingDeletes.isEmpty()) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index 891640843668..ee8690124c43 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -153,7 +153,7 @@ public boolean next() throws IOException { if (currentTask != null && !currentTask.isDataTask()) { String filePaths = referencedFiles(currentTask) - .map(file -> file.path().toString()) + .map(ContentFile::location) .collect(Collectors.joining(", ")); LOG.error("Error reading file(s): {}", filePaths, e); } @@ -199,7 +199,7 @@ private Map inputFiles() { } private EncryptedInputFile toEncryptedInputFile(ContentFile file) { - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); return EncryptedFiles.encryptedInput(inputFile, file.keyMetadata()); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index a2cb74c926c9..f45c152203ee 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -82,7 +82,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); // update the current file for Spark's filename() function diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index 25cd9eda6bce..a6e2891ae696 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -110,13 +110,13 @@ private CloseableIterable openChangelogScanTask(ChangelogScanTask t } CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); @@ -125,7 +125,7 @@ private CloseableIterable openDeletedDataFileScanTask(DeletedDataFi private CloseableIterable rows(ContentScanTask task, Schema readSchema) { Map idToConstant = constantsMap(task, readSchema); - String filePath = task.file().path().toString(); + String filePath = task.file().location(); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java index 7aa5a971562a..ee9449ee13c8 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java @@ -41,7 +41,7 @@ public EqualityDeleteRowReader( @Override protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter matches = - new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter(), true); + new SparkDeleteFilter(task.file().location(), task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = matches.requiredSchema(); @@ -49,7 +49,7 @@ protected CloseableIterator open(FileScanTask task) { DataFile file = task.file(); // update the current file for Spark's filename() function - InputFileBlockHolder.set(file.path().toString(), task.start(), task.length()); + InputFileBlockHolder.set(file.location(), task.start(), task.length()); return matches.findEqualityDeleteRows(open(task, requiredSchema, idToConstant)).iterator(); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java index 4b847474153c..1a894df29166 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java @@ -74,13 +74,13 @@ protected Stream> referencedFiles(PositionDeletesScanTask task) { @Override protected CloseableIterator open(PositionDeletesScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening position delete file {}", filePath); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with %s", task); // select out constant fields when pushing down filter to row reader diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 33b1d6275d6e..f24602fd5583 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -81,7 +81,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); SparkDeleteFilter deleteFilter = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); @@ -101,7 +101,7 @@ protected CloseableIterable open( if (task.isDataTask()) { return newDataIterable(task.asDataTask(), readSchema); } else { - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull( inputFile, "Could not find InputFile associated with FileScanTask"); return newIterable( diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java index a103a5003222..5f343128161d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java @@ -81,7 +81,7 @@ private static String taskInfo() { * @param files a list of files to delete */ public static void deleteFiles(String context, FileIO io, List> files) { - List paths = Lists.transform(files, file -> file.path().toString()); + List paths = Lists.transform(files, ContentFile::location); deletePaths(context, io, paths); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index 5a3963bc614c..7a6025b0731a 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -130,7 +130,7 @@ public void filter(Filter[] filters) { this.filteredLocations = fileLocations; List filteredTasks = tasks().stream() - .filter(file -> fileLocations.contains(file.file().path().toString())) + .filter(file -> fileLocations.contains(file.file().location())) .collect(Collectors.toList()); LOG.info( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java index c44bacf149b5..b241b8ed6363 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java @@ -60,7 +60,8 @@ public static void assertEquals(FileScanTask expected, FileScanTask actual) { } public static void assertEquals(DataFile expected, DataFile actual) { - Assert.assertEquals("Should match the serialized record path", expected.path(), actual.path()); + Assert.assertEquals( + "Should match the serialized record path", expected.location(), actual.location()); Assert.assertEquals( "Should match the serialized record format", expected.format(), actual.format()); Assert.assertEquals( @@ -105,7 +106,7 @@ private static List getFileScanTasksInFilePathOrder( ScanTaskGroup taskGroup) { return taskGroup.tasks().stream() // use file path + start position to differentiate the tasks - .sorted(Comparator.comparing(o -> o.file().path().toString() + "##" + o.start())) + .sorted(Comparator.comparing(o -> o.file().location() + "##" + o.start())) .collect(Collectors.toList()); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java index b0b3085dca70..dac09fea7562 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java @@ -42,7 +42,7 @@ public static List snapshotIds(Long... ids) { } public static List files(ContentFile... files) { - return Arrays.stream(files).map(file -> file.path().toString()).collect(Collectors.toList()); + return Arrays.stream(files).map(ContentFile::location).collect(Collectors.toList()); } public static void validateDataManifest( @@ -62,7 +62,7 @@ public static void validateDataManifest( actualDataSeqs.add(entry.dataSequenceNumber()); actualFileSeqs.add(entry.fileSequenceNumber()); actualSnapshotIds.add(entry.snapshotId()); - actualFiles.add(entry.file().path().toString()); + actualFiles.add(entry.file().location()); } assertSameElements("data seqs", actualDataSeqs, dataSeqs); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java index 82b36528996f..3e8953fb950c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java @@ -142,7 +142,7 @@ protected static String dbPath(String dbName) { } protected void withUnavailableFiles(Iterable> files, Action action) { - Iterable fileLocations = Iterables.transform(files, file -> file.path().toString()); + Iterable fileLocations = Iterables.transform(files, ContentFile::location); withUnavailableLocations(fileLocations, action); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java index 35dfb55d5bb2..0d523b659cc1 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java @@ -284,7 +284,7 @@ private void checkDelete(RowLevelOperationMode mode) throws Exception { } private int streamCount(DeleteFile deleteFile) { - CustomInputFile inputFile = INPUT_FILES.get(deleteFile.path().toString()); + CustomInputFile inputFile = INPUT_FILES.get(deleteFile.location()); return inputFile.streamCount(); } @@ -309,7 +309,7 @@ private List createAndInitTable(String operation, RowLevelOperationM List> posDeletes = dataFiles(table).stream() - .map(dataFile -> Pair.of(dataFile.path(), 0L)) + .map(dataFile -> Pair.of((CharSequence) dataFile.location(), 0L)) .collect(Collectors.toList()); Pair posDeleteResult = writePosDeletes(table, posDeletes); DeleteFile posDeleteFile = posDeleteResult.first(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 97843c152811..f4b270528bb1 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -204,7 +204,7 @@ public void dataFilesCleanupWithParallelTasks() { .forEach( file -> Assert.assertTrue( - "FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()))); + "FILE_A should be deleted", deletedFiles.contains(FILE_A.location()))); checkRemoveFilesResults(4L, 0, 0, 6L, 4L, 6, result); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 7a74329f3550..03ae730312fb 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -239,8 +239,8 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { Sets.newHashSet( "remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3")); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); - Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); + Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.location())); checkExpirationResults(2L, 0L, 0L, 3L, 3L, result); } @@ -552,7 +552,7 @@ public void testScanExpiredManifestInValidSnapshotAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -581,7 +581,7 @@ public void testScanExpiredManifestInValidSnapshotFastAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -624,7 +624,7 @@ public void testWithExpiringDanglingStageCommit() { .addedDataFiles(table.io()) .forEach( i -> { - expectedDeletes.add(i.path().toString()); + expectedDeletes.add(i.location()); }); // ManifestList should be deleted too @@ -693,7 +693,7 @@ public void testWithCherryPickTableSnapshot() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); @@ -742,7 +742,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); checkExpirationResults(0L, 0L, 0L, 1L, 1L, firstResult); @@ -762,7 +762,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); checkExpirationResults(0L, 0L, 0L, 0L, 2L, secondResult); @@ -863,7 +863,7 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted + FILE_A.location()), // deleted deletedFiles); checkExpirationResults(1, 0, 0, 2, 2, result); @@ -932,7 +932,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted + FILE_A.location()), // deleted deletedFiles); checkExpirationResults(1, 0, 0, 1, 2, result); @@ -1046,7 +1046,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back + FILE_B.location()), // added, but rolled back deletedFiles); checkExpirationResults(1, 0, 0, 1, 1, result); @@ -1095,9 +1095,9 @@ public void testExpireOlderThanWithDeleteFile() { secondSnapshot.manifestListLocation(), thirdSnapshot.manifestListLocation(), fourthSnapshot.manifestListLocation(), - FILE_A.path().toString(), - FILE_A_POS_DELETES.path().toString(), - FILE_A_EQ_DELETES.path().toString()); + FILE_A.location(), + FILE_A_POS_DELETES.location(), + FILE_A_EQ_DELETES.location()); expectedDeletes.addAll( thirdSnapshot.allManifests(table.io()).stream() @@ -1270,7 +1270,7 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe .withPartitionPath("c1=1") .withRecordCount(1) .build(); - dataFiles.add(df.path().toString()); + dataFiles.add(df.location()); table.newFastAppend().appendFile(df).commit(); } @@ -1343,9 +1343,9 @@ public void testExpireSomeCheckFilesDeleted() { // C, D should be retained (live) // B should be retained (previous snapshot points to it) // A should be deleted - Assert.assertTrue(deletedFiles.contains(FILE_A.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_B.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_C.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_D.path().toString())); + Assert.assertTrue(deletedFiles.contains(FILE_A.location())); + Assert.assertFalse(deletedFiles.contains(FILE_B.location())); + Assert.assertFalse(deletedFiles.contains(FILE_C.location())); + Assert.assertFalse(deletedFiles.contains(FILE_D.location())); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java index 8ac9be00ae90..dd330e461b1d 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -264,21 +264,21 @@ public void testPartitionedDeletesWithLesserSeqNo() { .collectAsList(); List> expected = ImmutableList.of( - Tuple2.apply(1L, FILE_B.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), - Tuple2.apply(3L, FILE_A2.path().toString()), - Tuple2.apply(3L, FILE_B2.path().toString()), - Tuple2.apply(3L, FILE_C2.path().toString()), - Tuple2.apply(3L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A_POS_DELETES.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); assertThat(actual).isEqualTo(expected); RemoveDanglingDeleteFiles.Result result = SparkActions.get().removeDanglingDeleteFiles(table).execute(); @@ -286,16 +286,16 @@ public void testPartitionedDeletesWithLesserSeqNo() { // because there are no data files in partition with a lesser sequence number Set removedDeleteFiles = StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) - .map(DeleteFile::path) + .map(DeleteFile::location) .collect(Collectors.toSet()); assertThat(removedDeleteFiles) .as("Expected 4 delete files removed") .hasSize(4) .containsExactlyInAnyOrder( - FILE_A_POS_DELETES.path(), - FILE_A2_POS_DELETES.path(), - FILE_A_EQ_DELETES.path(), - FILE_A2_EQ_DELETES.path()); + FILE_A_POS_DELETES.location(), + FILE_A2_POS_DELETES.location(), + FILE_A_EQ_DELETES.location(), + FILE_A2_EQ_DELETES.location()); List> actualAfter = spark .read() @@ -308,17 +308,17 @@ public void testPartitionedDeletesWithLesserSeqNo() { .collectAsList(); List> expectedAfter = ImmutableList.of( - Tuple2.apply(1L, FILE_B.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), - Tuple2.apply(3L, FILE_A2.path().toString()), - Tuple2.apply(3L, FILE_B2.path().toString()), - Tuple2.apply(3L, FILE_C2.path().toString()), - Tuple2.apply(3L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); assertThat(actualAfter).isEqualTo(expectedAfter); } @@ -354,21 +354,21 @@ public void testPartitionedDeletesWithEqSeqNo() { .collectAsList(); List> expected = ImmutableList.of( - Tuple2.apply(1L, FILE_A.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_C2.path().toString()), - Tuple2.apply(2L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A_POS_DELETES.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); assertThat(actual).isEqualTo(expected); RemoveDanglingDeleteFiles.Result result = SparkActions.get().removeDanglingDeleteFiles(table).execute(); @@ -376,12 +376,12 @@ public void testPartitionedDeletesWithEqSeqNo() { // because there are no data files in partition with a lesser sequence number Set removedDeleteFiles = StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) - .map(DeleteFile::path) + .map(DeleteFile::location) .collect(Collectors.toSet()); assertThat(removedDeleteFiles) .as("Expected two delete files removed") .hasSize(2) - .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.path(), FILE_B2_EQ_DELETES.path()); + .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.location(), FILE_B2_EQ_DELETES.location()); List> actualAfter = spark .read() @@ -394,19 +394,19 @@ public void testPartitionedDeletesWithEqSeqNo() { .collectAsList(); List> expectedAfter = ImmutableList.of( - Tuple2.apply(1L, FILE_A.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_C2.path().toString()), - Tuple2.apply(2L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A_POS_DELETES.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); assertThat(actualAfter).isEqualTo(expectedAfter); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 656b7358d1a5..490c7119304c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1983,8 +1983,7 @@ private void writeDF(Dataset df) { private List writePosDeletesToFile( Table table, DataFile dataFile, int outputDeleteFiles) { - return writePosDeletes( - table, dataFile.partition(), dataFile.path().toString(), outputDeleteFiles); + return writePosDeletes(table, dataFile.partition(), dataFile.location(), outputDeleteFiles); } private List writePosDeletes( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 5820d46703e2..9104cdadbb88 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -867,7 +867,7 @@ private void writePosDeletesForFiles( List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < deletesPerDataFile; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables @@ -904,17 +904,17 @@ private List deleteFiles(Table table) { private > List except(List first, List second) { Set secondPaths = - second.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + second.stream().map(ContentFile::location).collect(Collectors.toSet()); return first.stream() - .filter(f -> !secondPaths.contains(f.path().toString())) + .filter(f -> !secondPaths.contains(f.location())) .collect(Collectors.toList()); } private void assertNotContains(List original, List rewritten) { Set originalPaths = - original.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + original.stream().map(ContentFile::location).collect(Collectors.toSet()); Set rewrittenPaths = - rewritten.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + rewritten.stream().map(ContentFile::location).collect(Collectors.toSet()); rewrittenPaths.retainAll(originalPaths); Assert.assertEquals(0, rewrittenPaths.size()); } @@ -923,7 +923,7 @@ private void assertLocallySorted(List deleteFiles) { for (DeleteFile deleteFile : deleteFiles) { Dataset deletes = spark.read().format("iceberg").load("default." + TABLE_NAME + ".position_deletes"); - deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.path().toString())); + deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.location())); List rows = deletes.collectAsList(); Assert.assertFalse("Empty delete file found", rows.isEmpty()); int lastPos = 0; diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java index e5e0e350e50b..d417acdeb20a 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java @@ -114,7 +114,7 @@ public Boolean hasIterator(FileScanTask task) { } private String getKey(FileScanTask task) { - return task.file().path().toString(); + return task.file().location(); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java index 15325fe03c4e..724c6edde26a 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java @@ -168,7 +168,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { List manifestFiles = table.currentSnapshot().dataManifests(table.io()); try (ManifestReader reader = ManifestFiles.read(manifestFiles.get(0), table.io())) { DataFile file = reader.iterator().next(); - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -182,7 +182,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { try (ManifestReader reader = ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -196,7 +196,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { try (ManifestReader reader = ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 693bcec9eb05..b5c055925bf7 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -197,8 +197,8 @@ private void writeAndValidateWithLocations(Table table, File location, File expe Assert.assertTrue( String.format( "File should have the parent directory %s, but has: %s.", - expectedDataDir.getAbsolutePath(), dataFile.path()), - URI.create(dataFile.path().toString()) + expectedDataDir.getAbsolutePath(), dataFile.location()), + URI.create(dataFile.location()) .getPath() .startsWith(expectedDataDir.getAbsolutePath()))); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 77e1c373e8d9..013b8d4386af 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -110,7 +110,7 @@ public void testWriteFormatOptionOverridesTableProperties() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); Assert.assertEquals(FileFormat.PARQUET, fileFormat); }); } @@ -135,7 +135,7 @@ public void testNoWriteFormatOption() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); Assert.assertEquals(FileFormat.AVRO, fileFormat); }); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 84dff6666423..fd644cc49ca8 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -268,7 +268,7 @@ public void testEntriesTableDataFilePrune() { .select("data_file.file_path") .collectAsList()); - List singleExpected = ImmutableList.of(row(file.path())); + List singleExpected = ImmutableList.of(row(file.location())); assertEquals( "Should prune a single element from a nested struct", singleExpected, singleActual); @@ -307,7 +307,7 @@ public void testEntriesTableDataFilePruneMulti() { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a nested struct", multiExpected, multiActual); } @@ -341,7 +341,7 @@ public void testFilesSelectMap() { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a row", multiExpected, multiActual); } @@ -2315,7 +2315,7 @@ private DeleteFile writePosDeleteFile(Table table, long pos) { StructLike dataFilePartition = dataFile.partition(); PositionDelete delete = PositionDelete.create(); - delete.set(dataFile.path(), pos, null); + delete.set(dataFile.location(), pos, null); return writePositionDeletes(table, dataFileSpec, dataFilePartition, ImmutableList.of(delete)); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java index 2ec4f2f4f907..af3ec224cb4f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -26,6 +26,7 @@ import java.util.UUID; import java.util.function.BiFunction; import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -140,8 +141,8 @@ public void testNullRows() throws IOException { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); @@ -150,9 +151,9 @@ public void testNullRows() throws IOException { StructLikeSet actual = actual(tableName, tab); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); - StructLikeSet expected = - expected(tab, expectedDeletes, null, posDeletes.first().path().toString()); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); + StructLikeSet expected = expected(tab, expectedDeletes, null, posDeletes.first().location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -181,7 +182,7 @@ public void testPartitionedTable() throws IOException { GenericRecord partitionB = GenericRecord.create(tab.spec().partitionType()); partitionB.setField("data", "b"); StructLikeSet expected = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -221,7 +222,7 @@ public void testSelect() throws IOException { (delete, file) -> { int rowData = delete.get(2, GenericRecord.class).get(0, Integer.class); long pos = delete.get(1, Long.class); - return row(rowData, pos, file.path().toString(), file.path().toString()); + return row(rowData, pos, file.location(), file.location()); }; expected.addAll( deletesA.first().stream() @@ -270,7 +271,7 @@ public void testSplitTasks() throws IOException { List> deletes = Lists.newArrayList(); for (long i = 0; i < records; i++) { - deletes.add(positionDelete(tab.schema(), dFile.path(), i, (int) i, String.valueOf(i))); + deletes.add(positionDelete(tab.schema(), dFile.location(), i, (int) i, String.valueOf(i))); } DeleteFile posDeletes = FileHelpers.writePosDeleteFile( @@ -292,7 +293,7 @@ public void testSplitTasks() throws IOException { } StructLikeSet actual = actual(tableName, tab); - StructLikeSet expected = expected(tab, deletes, null, posDeletes.path().toString()); + StructLikeSet expected = expected(tab, deletes, null, posDeletes.location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -322,9 +323,9 @@ public void testPartitionFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data", "a"); Record partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTab.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -365,9 +366,9 @@ public void testPartitionTransformFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data_trunc", "a"); Record partitionB = partitionRecordTemplate.copy("data_trunc", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTable.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -415,7 +416,7 @@ public void testPartitionEvolutionReplace() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -427,7 +428,7 @@ public void testPartitionEvolutionReplace() throws Exception { deletes10.first(), partition10, tab.spec().specId(), - deletes10.second().path().toString()); + deletes10.second().location()); StructLikeSet actual10 = actual(tableName, tab, "partition.id = 10 AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expected10, actual10); @@ -465,7 +466,7 @@ public void testPartitionEvolutionAdd() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -477,7 +478,7 @@ public void testPartitionEvolutionAdd() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId0, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -520,7 +521,7 @@ public void testPartitionEvolutionRemove() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -532,7 +533,7 @@ public void testPartitionEvolutionRemove() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId1, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -577,7 +578,7 @@ public void testSpecIdFilter() throws Exception { deletesUnpartitioned.first(), partitionRecordTemplate, unpartitionedSpec, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, String.format("spec_id = %d", unpartitionedSpec)); Assert.assertEquals( @@ -589,9 +590,9 @@ public void testSpecIdFilter() throws Exception { StructLike partitionA = partitionRecordTemplate.copy("data", "a"); StructLike partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expected = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); expected.addAll( - expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().path().toString())); + expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().location())); StructLikeSet actual = actual(tableName, tab, String.format("spec_id = %d", dataSpec)); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); @@ -645,14 +646,14 @@ public void testSchemaEvolutionAdd() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); @@ -707,14 +708,14 @@ public void testSchemaEvolutionRemove() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); @@ -792,8 +793,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); @@ -827,7 +828,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { actual(tableName, tab, null, ImmutableList.of("file_path", "pos", "row", "spec_id")); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); StructLikeSet expected = expected(tab, expectedDeletes, null, null); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); @@ -846,8 +848,8 @@ public void testWriteMixedRows() throws Exception { // Add a delete file with row and without row List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dataFileA.path(), 0L)); - deletes.add(Pair.of(dataFileA.path(), 1L)); + deletes.add(Pair.of(dataFileA.location(), 0L)); + deletes.add(Pair.of(dataFileA.location(), 1L)); Pair deletesWithoutRow = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of("a"), deletes); @@ -905,7 +907,7 @@ public void testWriteMixedRows() throws Exception { expected( tab, Lists.newArrayList( - positionDelete(dataFileA.path(), 0L), positionDelete(dataFileA.path(), 1L)), + positionDelete(dataFileA.location(), 0L), positionDelete(dataFileA.location(), 1L)), partitionA, null)); allExpected.addAll(expected(tab, deletesWithRow.first(), partitionB, null)); @@ -1481,13 +1483,13 @@ private Pair>, DeleteFile> deleteFile( Lists.newArrayList( positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 0L, idPartition != null ? idPartition : 29, dataPartition != null ? dataPartition : "c"), positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 1L, idPartition != null ? idPartition : 61, dataPartition != null ? dataPartition : "r")); @@ -1533,9 +1535,9 @@ private void commit( Assert.assertEquals(expectedTargetFiles, addedFiles.size()); List sortedAddedFiles = - addedFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + addedFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); List sortedRewrittenFiles = - rewrittenFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + rewrittenFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); Assert.assertNotEquals("Lists should not be the same", sortedAddedFiles, sortedRewrittenFiles); baseTab diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java index edd4cdf08360..b09c995b30fa 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java @@ -470,7 +470,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set matchingFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().filter(filter).planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); matchingFileLocations.add(path); } } catch (IOException e) { @@ -480,7 +480,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set deletedFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); if (!matchingFileLocations.contains(path)) { io.deleteFile(path); deletedFileLocations.add(path); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index b894d32326dc..16fde3c95444 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -281,7 +281,7 @@ private void checkDeleteFile(DeleteFile expected, DeleteFile actual) { private void checkContentFile(ContentFile expected, ContentFile actual) { assertThat(actual.content()).isEqualTo(expected.content()); - assertThat(actual.path()).isEqualTo(expected.path()); + assertThat(actual.location()).isEqualTo(expected.location()); assertThat(actual.format()).isEqualTo(expected.format()); assertThat(actual.recordCount()).isEqualTo(expected.recordCount()); assertThat(actual.fileSizeInBytes()).isEqualTo(expected.fileSizeInBytes()); @@ -319,10 +319,10 @@ private DeleteFile createPositionDeleteFile(Table table, DataFile dataFile) { null, // no NaN counts ImmutableMap.of( MetadataColumns.DELETE_FILE_PATH.fieldId(), - Conversions.toByteBuffer(Types.StringType.get(), dataFile.path())), + Conversions.toByteBuffer(Types.StringType.get(), dataFile.location())), ImmutableMap.of( MetadataColumns.DELETE_FILE_PATH.fieldId(), - Conversions.toByteBuffer(Types.StringType.get(), dataFile.path())))) + Conversions.toByteBuffer(Types.StringType.get(), dataFile.location())))) .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) .build(); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index bbdd49b63df8..dda49b49465c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -343,10 +343,10 @@ public void testPosDeletesAllRowsInBatch() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -376,10 +376,10 @@ public void testPosDeletesWithDeletedColumn() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -455,8 +455,8 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 3L), // id = 89 - Pair.of(dataFile.path(), 5L) // id = 121 + Pair.of(dataFile.location(), 3L), // id = 89 + Pair.of(dataFile.location(), 5L) // id = 121 ); Pair posDeletes = @@ -486,10 +486,10 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { public void testFilterOnDeletedMetadataColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -611,13 +611,13 @@ public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOExceptio // Add positional deletes to the table List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 97L), - Pair.of(dataFile.path(), 98L), - Pair.of(dataFile.path(), 99L), - Pair.of(dataFile.path(), 101L), - Pair.of(dataFile.path(), 103L), - Pair.of(dataFile.path(), 107L), - Pair.of(dataFile.path(), 109L)); + Pair.of(dataFile.location(), 97L), + Pair.of(dataFile.location(), 98L), + Pair.of(dataFile.location(), 99L), + Pair.of(dataFile.location(), 101L), + Pair.of(dataFile.location(), 103L), + Pair.of(dataFile.location(), 107L), + Pair.of(dataFile.location(), 109L)); Pair posDeletes = FileHelpers.writeDeleteFile( table, diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index c58277ec084d..d3bbff5712ce 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -284,7 +284,7 @@ public void testExpireDeleteFiles() throws Exception { assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); - Path deleteFilePath = new Path(String.valueOf(deleteFile.location())); + Path deleteFilePath = new Path(deleteFile.location()); sql( "CALL %s.system.rewrite_data_files(" diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index e7698e5d5d07..57d4550d52a3 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -411,8 +411,7 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { assertThat(TestHelpers.deleteManifests(table)).as("Should have 1 delete manifest").hasSize(1); assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); - Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().location())); + Path deleteFilePath = new Path(TestHelpers.deleteFiles(table).iterator().next().location()); // wait to ensure files are old enough waitUntilAfter(System.currentTimeMillis()); diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 3b39847cd11d..f410fcc905fe 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -45,7 +45,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, 0.25, numDeleteFile); + writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } } diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index 3b41552052b0..b145a8178898 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -47,7 +47,7 @@ protected void appendData() throws IOException { // add pos-deletes table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, percentDeleteRow); + writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } } diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index c3823e88275e..b0439fe07ee8 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -48,7 +48,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { writePosDeletesWithNoise( - file.path(), + file.location(), NUM_ROWS, PERCENT_DELETE_ROW, (int) (percentUnrelatedDeletes / PERCENT_DELETE_ROW), diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index 01912c81ccdf..160dee9e7058 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -39,6 +39,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; @@ -647,7 +648,7 @@ private static void importUnpartitionedSparkTable( if (checkDuplicateFiles) { Dataset importedFiles = spark - .createDataset(Lists.transform(files, f -> f.path().toString()), Encoders.STRING()) + .createDataset(Lists.transform(files, ContentFile::location), Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -777,7 +778,7 @@ public static void importSparkPartitions( if (checkDuplicateFiles) { Dataset importedFiles = filesToImport - .map((MapFunction) f -> f.path().toString(), Encoders.STRING()) + .map((MapFunction) ContentFile::location, Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -796,7 +797,7 @@ public static void importSparkPartitions( .repartition(numShufflePartitions) .map( (MapFunction>) - file -> Tuple2.apply(file.path().toString(), file), + file -> Tuple2.apply(file.location(), file), Encoders.tuple(Encoders.STRING(), Encoders.javaSerialization(DataFile.class))) .orderBy(col("_1")) .mapPartitions( diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index c8e6f5679cd8..a6e2891ae696 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -116,7 +116,7 @@ CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { } private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); @@ -125,7 +125,7 @@ private CloseableIterable openDeletedDataFileScanTask(DeletedDataFi private CloseableIterable rows(ContentScanTask task, Schema readSchema) { Map idToConstant = constantsMap(task, readSchema); - String filePath = task.file().path().toString(); + String filePath = task.file().location(); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java index 7314043b15e2..dac09fea7562 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java @@ -42,7 +42,7 @@ public static List snapshotIds(Long... ids) { } public static List files(ContentFile... files) { - return Arrays.stream(files).map(file -> file.location()).collect(Collectors.toList()); + return Arrays.stream(files).map(ContentFile::location).collect(Collectors.toList()); } public static void validateDataManifest( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java index 86afd2f890ae..de68351f6e39 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java @@ -142,7 +142,7 @@ protected static String dbPath(String dbName) { } protected void withUnavailableFiles(Iterable> files, Action action) { - Iterable fileLocations = Iterables.transform(files, file -> file.location()); + Iterable fileLocations = Iterables.transform(files, ContentFile::location); withUnavailableLocations(fileLocations, action); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java index d9d7b78799ba..8114df1b5314 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java @@ -375,7 +375,7 @@ private List createAndInitTable(String operation, RowLevelOperationM List> posDeletes = dataFiles(table).stream() - .map(dataFile -> Pair.of(dataFile.path(), 0L)) + .map(dataFile -> Pair.of((CharSequence) dataFile.location(), 0L)) .collect(Collectors.toList()); Pair posDeleteResult = writePosDeletes(table, posDeletes); DeleteFile posDeleteFile = posDeleteResult.first(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index ffbe988e8d41..6a1eb0ed9054 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -874,7 +874,7 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()) // deleted + FILE_A.location()) // deleted ); checkExpirationResults(1, 0, 0, 2, 2, result); @@ -941,7 +941,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()) // deleted + FILE_A.location()) // deleted ); checkExpirationResults(1, 0, 0, 1, 2, result); } @@ -1053,7 +1053,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()) // added, but rolled back + FILE_B.location()) // added, but rolled back ); checkExpirationResults(1, 0, 0, 1, 1, result); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java index e58966cfea3f..ce86474c82f1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -313,21 +313,21 @@ public void testPartitionedDeletesWithLesserSeqNo() { .collectAsList(); List> expected = ImmutableList.of( - Tuple2.apply(1L, FILE_B.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileADeletes.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileA2Deletes.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileBDeletes.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileB2Deletes.path().toString()), - Tuple2.apply(3L, FILE_A2.path().toString()), - Tuple2.apply(3L, FILE_B2.path().toString()), - Tuple2.apply(3L, FILE_C2.path().toString()), - Tuple2.apply(3L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, fileADeletes.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, fileA2Deletes.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); RemoveDanglingDeleteFiles.Result result = @@ -338,16 +338,16 @@ public void testPartitionedDeletesWithLesserSeqNo() { Set removedDeleteFiles = StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) - .map(DeleteFile::path) + .map(DeleteFile::location) .collect(Collectors.toSet()); assertThat(removedDeleteFiles) .as("Expected 4 delete files removed") .hasSize(4) .containsExactlyInAnyOrder( - fileADeletes.path(), - fileA2Deletes.path(), - FILE_A_EQ_DELETES.path(), - FILE_A2_EQ_DELETES.path()); + fileADeletes.location(), + fileA2Deletes.location(), + FILE_A_EQ_DELETES.location(), + FILE_A2_EQ_DELETES.location()); List> actualAfter = spark @@ -361,17 +361,17 @@ public void testPartitionedDeletesWithLesserSeqNo() { .collectAsList(); List> expectedAfter = ImmutableList.of( - Tuple2.apply(1L, FILE_B.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileBDeletes.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileB2Deletes.path().toString()), - Tuple2.apply(3L, FILE_A2.path().toString()), - Tuple2.apply(3L, FILE_B2.path().toString()), - Tuple2.apply(3L, FILE_C2.path().toString()), - Tuple2.apply(3L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); assertThat(actualAfter).containsExactlyInAnyOrderElementsOf(expectedAfter); } @@ -414,21 +414,21 @@ public void testPartitionedDeletesWithEqSeqNo() { .collectAsList(); List> expected = ImmutableList.of( - Tuple2.apply(1L, FILE_A.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileADeletes.path().toString()), - Tuple2.apply(2L, FILE_A2.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileA2Deletes.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileBDeletes.path().toString()), - Tuple2.apply(2L, FILE_B2.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileB2Deletes.path().toString()), - Tuple2.apply(2L, FILE_C2.path().toString()), - Tuple2.apply(2L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, fileADeletes.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, fileA2Deletes.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); RemoveDanglingDeleteFiles.Result result = @@ -438,12 +438,12 @@ public void testPartitionedDeletesWithEqSeqNo() { // because there are no data files in partition with a lesser sequence number Set removedDeleteFiles = StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) - .map(DeleteFile::path) + .map(DeleteFile::location) .collect(Collectors.toSet()); assertThat(removedDeleteFiles) .as("Expected two delete files removed") .hasSize(2) - .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.path(), FILE_B2_EQ_DELETES.path()); + .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.location(), FILE_B2_EQ_DELETES.location()); List> actualAfter = spark @@ -457,19 +457,19 @@ public void testPartitionedDeletesWithEqSeqNo() { .collectAsList(); List> expectedAfter = ImmutableList.of( - Tuple2.apply(1L, FILE_A.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileADeletes.path().toString()), - Tuple2.apply(2L, FILE_A2.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileA2Deletes.path().toString()), - Tuple2.apply(2L, fileBDeletes.path().toString()), - Tuple2.apply(2L, FILE_B2.path().toString()), - Tuple2.apply(2L, fileB2Deletes.path().toString()), - Tuple2.apply(2L, FILE_C2.path().toString()), - Tuple2.apply(2L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, fileADeletes.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, fileA2Deletes.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); assertThat(actualAfter).containsExactlyInAnyOrderElementsOf(expectedAfter); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 38c4d32a90d2..bd307a468279 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -2152,8 +2152,7 @@ private void writeDF(Dataset df) { private List writePosDeletesToFile( Table table, DataFile dataFile, int outputDeleteFiles) { - return writePosDeletes( - table, dataFile.partition(), dataFile.path().toString(), outputDeleteFiles); + return writePosDeletes(table, dataFile.partition(), dataFile.location(), outputDeleteFiles); } private List writePosDeletes( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 12b104fca27c..3bce399b34a9 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -898,7 +898,7 @@ private void writePosDeletesForFiles( List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < deletesPerDataFile; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables @@ -936,17 +936,17 @@ private List deleteFiles(Table table) { private > List except(List first, List second) { Set secondPaths = - second.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + second.stream().map(ContentFile::location).collect(Collectors.toSet()); return first.stream() - .filter(f -> !secondPaths.contains(f.path().toString())) + .filter(f -> !secondPaths.contains(f.location())) .collect(Collectors.toList()); } private void assertNotContains(List original, List rewritten) { Set originalPaths = - original.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + original.stream().map(ContentFile::location).collect(Collectors.toSet()); Set rewrittenPaths = - rewritten.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + rewritten.stream().map(ContentFile::location).collect(Collectors.toSet()); rewrittenPaths.retainAll(originalPaths); assertThat(rewrittenPaths).hasSize(0); } @@ -955,7 +955,7 @@ private void assertLocallySorted(List deleteFiles) { for (DeleteFile deleteFile : deleteFiles) { Dataset deletes = spark.read().format("iceberg").load("default." + TABLE_NAME + ".position_deletes"); - deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.path().toString())); + deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.location())); List rows = deletes.collectAsList(); assertThat(rows).as("Empty delete file found").isNotEmpty(); int lastPos = 0; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index dcd347a34f81..7d38cf55c98c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -266,7 +266,7 @@ public void testEntriesTableDataFilePrune() { .select("data_file.file_path") .collectAsList()); - List singleExpected = ImmutableList.of(row(file.path())); + List singleExpected = ImmutableList.of(row(file.location())); assertEquals( "Should prune a single element from a nested struct", singleExpected, singleActual); @@ -305,7 +305,7 @@ public void testEntriesTableDataFilePruneMulti() { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a nested struct", multiExpected, multiActual); } @@ -339,7 +339,7 @@ public void testFilesSelectMap() { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a row", multiExpected, multiActual); } @@ -2341,7 +2341,7 @@ private DeleteFile writePosDeleteFile(Table table, long pos) { StructLike dataFilePartition = dataFile.partition(); PositionDelete delete = PositionDelete.create(); - delete.set(dataFile.path(), pos, null); + delete.set(dataFile.location(), pos, null); return writePositionDeletes(table, dataFileSpec, dataFilePartition, ImmutableList.of(delete)); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java index 983e52cdf897..357cd0cdaa06 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -30,6 +30,7 @@ import java.util.UUID; import java.util.function.BiFunction; import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -1586,9 +1587,9 @@ private void commit( assertThat(addedFiles).hasSize(expectedTargetFiles); List sortedAddedFiles = - addedFiles.stream().map(f -> f.location()).sorted().collect(Collectors.toList()); + addedFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); List sortedRewrittenFiles = - rewrittenFiles.stream().map(f -> f.location()).sorted().collect(Collectors.toList()); + rewrittenFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); assertThat(sortedRewrittenFiles) .as("Lists should not be the same") .isNotEqualTo(sortedAddedFiles); From fe2f593cd025223e4ab5ab41a296fb106ce3b1cf Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 11 Dec 2024 21:45:52 +0100 Subject: [PATCH 243/313] Infra: Build Iceberg REST fixture docker image for `arm64` architecture (#11753) --- .../publish-iceberg-rest-fixture-docker.yml | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/.github/workflows/publish-iceberg-rest-fixture-docker.yml b/.github/workflows/publish-iceberg-rest-fixture-docker.yml index eb946163fdff..4e9aa53e1a27 100644 --- a/.github/workflows/publish-iceberg-rest-fixture-docker.yml +++ b/.github/workflows/publish-iceberg-rest-fixture-docker.yml @@ -49,8 +49,15 @@ jobs: if: github.event_name == 'push' && contains(github.ref, 'refs/tags/') run: | echo "DOCKER_IMAGE_VERSION=`echo ${{ github.ref }} | tr -d -c 0-9.`" >> "$GITHUB_ENV" - - name: Build Docker Image - run: docker build -t $DOCKER_REPOSITORY/$DOCKER_IMAGE_TAG:$DOCKER_IMAGE_VERSION -f docker/iceberg-rest-fixture/Dockerfile . - - name: Push Docker Image - run: | - docker push $DOCKER_REPOSITORY/$DOCKER_IMAGE_TAG:$DOCKER_IMAGE_VERSION + - name: Set up QEMU + uses: docker/setup-qemu-action@v3 + - name: Set up Docker Buildx + uses: docker/setup-buildx-action@v3 + - name: Build and Push + uses: docker/build-push-action@v6 + with: + context: ./ + file: ./docker/iceberg-rest-fixture/Dockerfile + platforms: linux/amd64,linux/arm64 + push: true + tags: ${{ env.DOCKER_REPOSITORY }}/${{ env.DOCKER_IMAGE_TAG }}:${{ env.DOCKER_IMAGE_VERSION }} From af5e156edec0b225b20cb81662abd491c6d289dd Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 12 Dec 2024 15:50:21 +0800 Subject: [PATCH 244/313] Docs: fix typos in spec (#11759) --- format/spec.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/format/spec.md b/format/spec.md index c9cef3290850..728453f86ba9 100644 --- a/format/spec.md +++ b/format/spec.md @@ -441,7 +441,7 @@ Partition specs capture the transform from table data to partition values. This Partition fields that use an unknown transform can be read by ignoring the partition field for the purpose of filtering data files during scan planning. In v1 and v2, readers should ignore fields with unknown transforms while reading; this behavior is required in v3. Writers are not allowed to commit data using a partition spec that contains a field with an unknown transform. -Two partition specs are considered equivalent with each other if they have the same number of fields and for each corresponding field, the fields have the same source column IDs, transform definition and partition name. Writers must not create a new parition spec if there already exists a compatible partition spec defined in the table. +Two partition specs are considered equivalent with each other if they have the same number of fields and for each corresponding field, the fields have the same source column IDs, transform definition and partition name. Writers must not create a new partition spec if there already exists a compatible partition spec defined in the table. Partition field IDs must be reused if an existing partition spec contains an equivalent field. @@ -792,7 +792,7 @@ Notes: 1. An alternative, *strict projection*, creates a partition predicate that will match a file if all of the rows in the file must match the scan predicate. These projections are used to calculate the residual predicates for each file in a scan. 2. For example, if `file_a` has rows with `id` between 1 and 10 and a delete file contains rows with `id` between 1 and 4, a scan for `id = 9` may ignore the delete file because none of the deletes can match a row that will be selected. -3. Floating point partition values are considered equal if their IEEE 754 floating-point "single format" bit layout are equal with NaNs normalized to have only the the most significant mantissa bit set (the equivelant of calling `Float.floatToIntBits` or `Double.doubleToLongBits` in Java). The Avro specification requires all floating point values to be encoded in this format. +3. Floating point partition values are considered equal if their IEEE 754 floating-point "single format" bit layout are equal with NaNs normalized to have only the the most significant mantissa bit set (the equivalent of calling `Float.floatToIntBits` or `Double.doubleToLongBits` in Java). The Avro specification requires all floating point values to be encoded in this format. 4. Unknown partition transforms do not affect partition equality. Although partition fields with unknown transforms are ignored for filtering, the result of an unknown transform is still used when testing whether partition values are equal. ### Snapshot References From 587620b20dcdada7b1fa730206937c4b6a7ce527 Mon Sep 17 00:00:00 2001 From: Ppei-Wang Date: Thu, 12 Dec 2024 15:52:02 +0800 Subject: [PATCH 245/313] Spark 3.4,3.5: Fix issue when views group by an ordinal (#11729) --- .../sql/catalyst/analysis/ResolveViews.scala | 4 +-- .../iceberg/spark/extensions/TestViews.java | 34 +++++++++++++++++++ .../sql/catalyst/analysis/ResolveViews.scala | 4 +-- .../iceberg/spark/extensions/TestViews.java | 34 +++++++++++++++++++ 4 files changed, 72 insertions(+), 4 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index 3c0fd6d90f62..37791dbbec2b 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -123,10 +123,10 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look private def rewriteIdentifiers( plan: LogicalPlan, catalogAndNamespace: Seq[String]): LogicalPlan = { - // Substitute CTEs within the view, then rewrite unresolved functions and relations + // Substitute CTEs and Unresolved Ordinals within the view, then rewrite unresolved functions and relations qualifyTableIdentifiers( qualifyFunctionIdentifiers( - CTESubstitution.apply(plan), + SubstituteUnresolvedOrdinals.apply(CTESubstitution.apply(plan)), catalogAndNamespace), catalogAndNamespace) } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index 1380711ed7b7..51d6064d9a14 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -533,6 +533,40 @@ public void readFromViewWithCTE() throws NoSuchTableException { assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(10, 1L)); } + @Test + public void readFromViewWithGroupByOrdinal() throws NoSuchTableException { + insertRows(3); + insertRows(2); + String viewName = viewName("viewWithGroupByOrdinal"); + String sql = String.format("SELECT id, count(1) FROM %s GROUP BY 1", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1, 2L), row(2, 2L), row(3, 1L)); + } + + @Test + public void createViewWithGroupByOrdinal() throws NoSuchTableException { + insertRows(3); + insertRows(2); + String viewName = viewName("createViewWithGroupByOrdinal"); + sql("CREATE VIEW %s AS SELECT id, count(1) FROM %s GROUP BY 1", viewName, tableName); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1, 2L), row(2, 2L), row(3, 1L)); + } + @Test public void rewriteFunctionIdentifier() { String viewName = viewName("rewriteFunctionIdentifier"); diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index 5589a9c4bb17..c80ce448e051 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -124,10 +124,10 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look private def rewriteIdentifiers( plan: LogicalPlan, catalogAndNamespace: Seq[String]): LogicalPlan = { - // Substitute CTEs within the view, then rewrite unresolved functions and relations + // Substitute CTEs and Unresolved Ordinals within the view, then rewrite unresolved functions and relations qualifyTableIdentifiers( qualifyFunctionIdentifiers( - CTESubstitution.apply(plan), + SubstituteUnresolvedOrdinals.apply(CTESubstitution.apply(plan)), catalogAndNamespace), catalogAndNamespace) } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index a6149a0ae49e..03a31d44d69d 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -562,6 +562,40 @@ public void readFromViewWithCTE() throws NoSuchTableException { assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(10, 1L)); } + @TestTemplate + public void readFromViewWithGroupByOrdinal() throws NoSuchTableException { + insertRows(3); + insertRows(2); + String viewName = viewName("viewWithGroupByOrdinal"); + String sql = String.format("SELECT id, count(1) FROM %s GROUP BY 1", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1, 2L), row(2, 2L), row(3, 1L)); + } + + @TestTemplate + public void createViewWithGroupByOrdinal() throws NoSuchTableException { + insertRows(3); + insertRows(2); + String viewName = viewName("createViewWithGroupByOrdinal"); + sql("CREATE VIEW %s AS SELECT id, count(1) FROM %s GROUP BY 1", viewName, tableName); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1, 2L), row(2, 2L), row(3, 1L)); + } + @TestTemplate public void rewriteFunctionIdentifier() { assumeThat(catalogName) From 5c00b29ae29b2c5f763559c70638e8e76431ecfd Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Thu, 12 Dec 2024 14:31:21 +0530 Subject: [PATCH 246/313] Spark: Remove deprecated SparkAppenderFactory (#11727) --- .../spark/source/WritersBenchmark.java | 97 ------ .../spark/source/SparkAppenderFactory.java | 322 ----------------- .../source/TestSparkAppenderFactory.java | 64 ---- .../spark/source/TestSparkMergingMetrics.java | 70 ---- .../spark/source/WritersBenchmark.java | 97 ------ .../spark/source/SparkAppenderFactory.java | 328 ------------------ .../source/TestSparkAppenderFactory.java | 64 ---- .../spark/source/TestSparkMergingMetrics.java | 70 ---- .../spark/source/WritersBenchmark.java | 97 ------ .../spark/source/SparkAppenderFactory.java | 328 ------------------ .../source/TestSparkAppenderFactory.java | 64 ---- .../spark/source/TestSparkMergingMetrics.java | 71 ---- 12 files changed, 1672 deletions(-) delete mode 100644 spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java delete mode 100644 spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java delete mode 100644 spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java delete mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java delete mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java delete mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java delete mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java delete mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java index dd2cf80716f6..a6fe97914216 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java @@ -41,8 +41,6 @@ import org.apache.iceberg.io.FanoutDataWriter; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -149,33 +147,6 @@ public void writeUnpartitionedClusteredDataWriter(Blackhole blackhole) throws IO blackhole.consume(writer); } - @Benchmark - @Threads(1) - public void writeUnpartitionedLegacyDataWriter(Blackhole blackhole) throws IOException { - FileIO io = table().io(); - - OutputFileFactory fileFactory = newFileFactory(); - - Schema writeSchema = table().schema(); - StructType sparkWriteType = SparkSchemaUtil.convert(writeSchema); - SparkAppenderFactory appenders = - SparkAppenderFactory.builderFor(table(), writeSchema, sparkWriteType) - .spec(unpartitionedSpec) - .build(); - - TaskWriter writer = - new UnpartitionedWriter<>( - unpartitionedSpec, fileFormat(), appenders, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES); - - try (TaskWriter closableWriter = writer) { - for (InternalRow row : rows) { - closableWriter.write(row); - } - } - - blackhole.consume(writer.complete()); - } - @Benchmark @Threads(1) public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOException { @@ -206,40 +177,6 @@ public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOEx blackhole.consume(writer); } - @Benchmark - @Threads(1) - public void writePartitionedLegacyDataWriter(Blackhole blackhole) throws IOException { - FileIO io = table().io(); - - OutputFileFactory fileFactory = newFileFactory(); - - Schema writeSchema = table().schema(); - StructType sparkWriteType = SparkSchemaUtil.convert(writeSchema); - SparkAppenderFactory appenders = - SparkAppenderFactory.builderFor(table(), writeSchema, sparkWriteType) - .spec(partitionedSpec) - .build(); - - TaskWriter writer = - new SparkPartitionedWriter( - partitionedSpec, - fileFormat(), - appenders, - fileFactory, - io, - TARGET_FILE_SIZE_IN_BYTES, - writeSchema, - sparkWriteType); - - try (TaskWriter closableWriter = writer) { - for (InternalRow row : rows) { - closableWriter.write(row); - } - } - - blackhole.consume(writer.complete()); - } - @Benchmark @Threads(1) public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOException { @@ -270,40 +207,6 @@ public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOExcep blackhole.consume(writer); } - @Benchmark - @Threads(1) - public void writePartitionedLegacyFanoutDataWriter(Blackhole blackhole) throws IOException { - FileIO io = table().io(); - - OutputFileFactory fileFactory = newFileFactory(); - - Schema writeSchema = table().schema(); - StructType sparkWriteType = SparkSchemaUtil.convert(writeSchema); - SparkAppenderFactory appenders = - SparkAppenderFactory.builderFor(table(), writeSchema, sparkWriteType) - .spec(partitionedSpec) - .build(); - - TaskWriter writer = - new SparkPartitionedFanoutWriter( - partitionedSpec, - fileFormat(), - appenders, - fileFactory, - io, - TARGET_FILE_SIZE_IN_BYTES, - writeSchema, - sparkWriteType); - - try (TaskWriter closableWriter = writer) { - for (InternalRow row : rows) { - closableWriter.write(row); - } - } - - blackhole.consume(writer.complete()); - } - @Benchmark @Threads(1) public void writePartitionedClusteredEqualityDeleteWriter(Blackhole blackhole) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java deleted file mode 100644 index 2ff4f9156273..000000000000 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ /dev/null @@ -1,322 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Map; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.deletes.EqualityDeleteWriter; -import org.apache.iceberg.deletes.PositionDeleteWriter; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.exceptions.RuntimeIOException; -import org.apache.iceberg.io.DataWriter; -import org.apache.iceberg.io.DeleteSchemaUtil; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.data.SparkAvroWriter; -import org.apache.iceberg.spark.data.SparkOrcWriter; -import org.apache.iceberg.spark.data.SparkParquetWriters; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.unsafe.types.UTF8String; - -/** - * @deprecated since 1.7.0, will be removed in 1.8.0; use {@link SparkFileWriterFactory} instead. - */ -@Deprecated -class SparkAppenderFactory implements FileAppenderFactory { - private final Map properties; - private final Schema writeSchema; - private final StructType dsSchema; - private final PartitionSpec spec; - private final int[] equalityFieldIds; - private final Schema eqDeleteRowSchema; - private final Schema posDeleteRowSchema; - - private StructType eqDeleteSparkType = null; - private StructType posDeleteSparkType = null; - - SparkAppenderFactory( - Map properties, - Schema writeSchema, - StructType dsSchema, - PartitionSpec spec, - int[] equalityFieldIds, - Schema eqDeleteRowSchema, - Schema posDeleteRowSchema) { - this.properties = properties; - this.writeSchema = writeSchema; - this.dsSchema = dsSchema; - this.spec = spec; - this.equalityFieldIds = equalityFieldIds; - this.eqDeleteRowSchema = eqDeleteRowSchema; - this.posDeleteRowSchema = posDeleteRowSchema; - } - - static Builder builderFor(Table table, Schema writeSchema, StructType dsSchema) { - return new Builder(table, writeSchema, dsSchema); - } - - static class Builder { - private final Table table; - private final Schema writeSchema; - private final StructType dsSchema; - private PartitionSpec spec; - private int[] equalityFieldIds; - private Schema eqDeleteRowSchema; - private Schema posDeleteRowSchema; - - Builder(Table table, Schema writeSchema, StructType dsSchema) { - this.table = table; - this.spec = table.spec(); - this.writeSchema = writeSchema; - this.dsSchema = dsSchema; - } - - Builder spec(PartitionSpec newSpec) { - this.spec = newSpec; - return this; - } - - Builder equalityFieldIds(int[] newEqualityFieldIds) { - this.equalityFieldIds = newEqualityFieldIds; - return this; - } - - Builder eqDeleteRowSchema(Schema newEqDeleteRowSchema) { - this.eqDeleteRowSchema = newEqDeleteRowSchema; - return this; - } - - Builder posDelRowSchema(Schema newPosDelRowSchema) { - this.posDeleteRowSchema = newPosDelRowSchema; - return this; - } - - SparkAppenderFactory build() { - Preconditions.checkNotNull(table, "Table must not be null"); - Preconditions.checkNotNull(writeSchema, "Write Schema must not be null"); - Preconditions.checkNotNull(dsSchema, "DS Schema must not be null"); - if (equalityFieldIds != null) { - Preconditions.checkNotNull( - eqDeleteRowSchema, - "Equality Field Ids and Equality Delete Row Schema" + " must be set together"); - } - if (eqDeleteRowSchema != null) { - Preconditions.checkNotNull( - equalityFieldIds, - "Equality Field Ids and Equality Delete Row Schema" + " must be set together"); - } - - return new SparkAppenderFactory( - table.properties(), - writeSchema, - dsSchema, - spec, - equalityFieldIds, - eqDeleteRowSchema, - posDeleteRowSchema); - } - } - - private StructType lazyEqDeleteSparkType() { - if (eqDeleteSparkType == null) { - Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); - this.eqDeleteSparkType = SparkSchemaUtil.convert(eqDeleteRowSchema); - } - return eqDeleteSparkType; - } - - private StructType lazyPosDeleteSparkType() { - if (posDeleteSparkType == null) { - Preconditions.checkNotNull( - posDeleteRowSchema, "Position delete row schema shouldn't be null"); - this.posDeleteSparkType = SparkSchemaUtil.convert(posDeleteRowSchema); - } - return posDeleteSparkType; - } - - @Override - public FileAppender newAppender(OutputFile file, FileFormat fileFormat) { - MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties); - try { - switch (fileFormat) { - case PARQUET: - return Parquet.write(file) - .createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dsSchema, msgType)) - .setAll(properties) - .metricsConfig(metricsConfig) - .schema(writeSchema) - .overwrite() - .build(); - - case AVRO: - return Avro.write(file) - .createWriterFunc(ignored -> new SparkAvroWriter(dsSchema)) - .setAll(properties) - .schema(writeSchema) - .overwrite() - .build(); - - case ORC: - return ORC.write(file) - .createWriterFunc(SparkOrcWriter::new) - .setAll(properties) - .metricsConfig(metricsConfig) - .schema(writeSchema) - .overwrite() - .build(); - - default: - throw new UnsupportedOperationException("Cannot write unknown format: " + fileFormat); - } - } catch (IOException e) { - throw new RuntimeIOException(e); - } - } - - @Override - public DataWriter newDataWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - return new DataWriter<>( - newAppender(file.encryptingOutputFile(), format), - format, - file.encryptingOutputFile().location(), - spec, - partition, - file.keyMetadata()); - } - - @Override - public EqualityDeleteWriter newEqDeleteWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - Preconditions.checkState( - equalityFieldIds != null && equalityFieldIds.length > 0, - "Equality field ids shouldn't be null or empty when creating equality-delete writer"); - Preconditions.checkNotNull( - eqDeleteRowSchema, - "Equality delete row schema shouldn't be null when creating equality-delete writer"); - - try { - switch (format) { - case PARQUET: - return Parquet.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(lazyEqDeleteSparkType(), msgType)) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - case AVRO: - return Avro.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc(ignored -> new SparkAvroWriter(lazyEqDeleteSparkType())) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - case ORC: - return ORC.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc(SparkOrcWriter::new) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write equality-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException("Failed to create new equality delete writer", e); - } - } - - @Override - public PositionDeleteWriter newPosDeleteWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - try { - switch (format) { - case PARQUET: - StructType sparkPosDeleteSchema = - SparkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return Parquet.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(sparkPosDeleteSchema, msgType)) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .transformPaths(path -> UTF8String.fromString(path.toString())) - .buildPositionWriter(); - - case AVRO: - return Avro.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc(ignored -> new SparkAvroWriter(lazyPosDeleteSparkType())) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .buildPositionWriter(); - - case ORC: - return ORC.writeDeletes(file.encryptingOutputFile()) - .createWriterFunc(SparkOrcWriter::new) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .transformPaths(path -> UTF8String.fromString(path.toString())) - .buildPositionWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write pos-deletes for unsupported file format: " + format); - } - - } catch (IOException e) { - throw new UncheckedIOException("Failed to create new equality delete writer", e); - } - } -} diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java deleted file mode 100644 index 0664400c7911..000000000000 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import java.util.List; -import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.TestAppenderFactory; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.unsafe.types.UTF8String; - -public class TestSparkAppenderFactory extends TestAppenderFactory { - - private final StructType sparkType = SparkSchemaUtil.convert(SCHEMA); - - @Override - protected FileAppenderFactory createAppenderFactory( - List equalityFieldIds, Schema eqDeleteSchema, Schema posDeleteRowSchema) { - return SparkAppenderFactory.builderFor(table, table.schema(), sparkType) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .eqDeleteRowSchema(eqDeleteSchema) - .posDelRowSchema(posDeleteRowSchema) - .build(); - } - - @Override - protected InternalRow createRow(Integer id, String data) { - InternalRow row = new GenericInternalRow(2); - row.update(0, id); - row.update(1, UTF8String.fromString(data)); - return row; - } - - @Override - protected StructLikeSet expectedRowSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - for (InternalRow row : rows) { - InternalRowWrapper wrapper = new InternalRowWrapper(sparkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java deleted file mode 100644 index fa1ff9fbfde0..000000000000 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestMergingMetrics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.catalyst.InternalRow; - -public class TestSparkMergingMetrics extends TestMergingMetrics { - - @Override - protected FileAppender writeAndGetAppender(List records) throws IOException { - Table testTable = - new BaseTable(null, "dummy") { - @Override - public Map properties() { - return Collections.emptyMap(); - } - - @Override - public SortOrder sortOrder() { - return SortOrder.unsorted(); - } - - @Override - public PartitionSpec spec() { - return PartitionSpec.unpartitioned(); - } - }; - - File tempFile = File.createTempFile("junit", null, tempDir); - FileAppender appender = - SparkAppenderFactory.builderFor(testTable, SCHEMA, SparkSchemaUtil.convert(SCHEMA)) - .build() - .newAppender(org.apache.iceberg.Files.localOutput(tempFile), fileFormat); - try (FileAppender fileAppender = appender) { - records.stream() - .map(r -> new StructInternalRow(SCHEMA.asStruct()).setStruct(r)) - .forEach(fileAppender::add); - } - return appender; - } -} diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java index 963159fe4ee9..06efb6ba20e4 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java @@ -44,8 +44,6 @@ import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -176,33 +174,6 @@ public void writeUnpartitionedClusteredDataWriter(Blackhole blackhole) throws IO blackhole.consume(writer); } - @Benchmark - @Threads(1) - public void writeUnpartitionedLegacyDataWriter(Blackhole blackhole) throws IOException { - FileIO io = table().io(); - - OutputFileFactory fileFactory = newFileFactory(); - - Schema writeSchema = table().schema(); - StructType sparkWriteType = SparkSchemaUtil.convert(writeSchema); - SparkAppenderFactory appenders = - SparkAppenderFactory.builderFor(table(), writeSchema, sparkWriteType) - .spec(unpartitionedSpec) - .build(); - - TaskWriter writer = - new UnpartitionedWriter<>( - unpartitionedSpec, fileFormat(), appenders, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES); - - try (TaskWriter closableWriter = writer) { - for (InternalRow row : rows) { - closableWriter.write(row); - } - } - - blackhole.consume(writer.complete()); - } - @Benchmark @Threads(1) public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOException { @@ -233,40 +204,6 @@ public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOEx blackhole.consume(writer); } - @Benchmark - @Threads(1) - public void writePartitionedLegacyDataWriter(Blackhole blackhole) throws IOException { - FileIO io = table().io(); - - OutputFileFactory fileFactory = newFileFactory(); - - Schema writeSchema = table().schema(); - StructType sparkWriteType = SparkSchemaUtil.convert(writeSchema); - SparkAppenderFactory appenders = - SparkAppenderFactory.builderFor(table(), writeSchema, sparkWriteType) - .spec(partitionedSpec) - .build(); - - TaskWriter writer = - new SparkPartitionedWriter( - partitionedSpec, - fileFormat(), - appenders, - fileFactory, - io, - TARGET_FILE_SIZE_IN_BYTES, - writeSchema, - sparkWriteType); - - try (TaskWriter closableWriter = writer) { - for (InternalRow row : rows) { - closableWriter.write(row); - } - } - - blackhole.consume(writer.complete()); - } - @Benchmark @Threads(1) public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOException { @@ -297,40 +234,6 @@ public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOExcep blackhole.consume(writer); } - @Benchmark - @Threads(1) - public void writePartitionedLegacyFanoutDataWriter(Blackhole blackhole) throws IOException { - FileIO io = table().io(); - - OutputFileFactory fileFactory = newFileFactory(); - - Schema writeSchema = table().schema(); - StructType sparkWriteType = SparkSchemaUtil.convert(writeSchema); - SparkAppenderFactory appenders = - SparkAppenderFactory.builderFor(table(), writeSchema, sparkWriteType) - .spec(partitionedSpec) - .build(); - - TaskWriter writer = - new SparkPartitionedFanoutWriter( - partitionedSpec, - fileFormat(), - appenders, - fileFactory, - io, - TARGET_FILE_SIZE_IN_BYTES, - writeSchema, - sparkWriteType); - - try (TaskWriter closableWriter = writer) { - for (InternalRow row : rows) { - closableWriter.write(row); - } - } - - blackhole.consume(writer.complete()); - } - @Benchmark @Threads(1) public void writePartitionedClusteredEqualityDeleteWriter(Blackhole blackhole) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java deleted file mode 100644 index c822ed743f85..000000000000 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ /dev/null @@ -1,328 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Map; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.deletes.EqualityDeleteWriter; -import org.apache.iceberg.deletes.PositionDeleteWriter; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.encryption.EncryptionUtil; -import org.apache.iceberg.exceptions.RuntimeIOException; -import org.apache.iceberg.io.DataWriter; -import org.apache.iceberg.io.DeleteSchemaUtil; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.data.SparkAvroWriter; -import org.apache.iceberg.spark.data.SparkOrcWriter; -import org.apache.iceberg.spark.data.SparkParquetWriters; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.unsafe.types.UTF8String; - -/** - * @deprecated since 1.7.0, will be removed in 1.8.0; use {@link SparkFileWriterFactory} instead. - */ -@Deprecated -class SparkAppenderFactory implements FileAppenderFactory { - private final Map properties; - private final Schema writeSchema; - private final StructType dsSchema; - private final PartitionSpec spec; - private final int[] equalityFieldIds; - private final Schema eqDeleteRowSchema; - private final Schema posDeleteRowSchema; - - private StructType eqDeleteSparkType = null; - private StructType posDeleteSparkType = null; - - SparkAppenderFactory( - Map properties, - Schema writeSchema, - StructType dsSchema, - PartitionSpec spec, - int[] equalityFieldIds, - Schema eqDeleteRowSchema, - Schema posDeleteRowSchema) { - this.properties = properties; - this.writeSchema = writeSchema; - this.dsSchema = dsSchema; - this.spec = spec; - this.equalityFieldIds = equalityFieldIds; - this.eqDeleteRowSchema = eqDeleteRowSchema; - this.posDeleteRowSchema = posDeleteRowSchema; - } - - static Builder builderFor(Table table, Schema writeSchema, StructType dsSchema) { - return new Builder(table, writeSchema, dsSchema); - } - - static class Builder { - private final Table table; - private final Schema writeSchema; - private final StructType dsSchema; - private PartitionSpec spec; - private int[] equalityFieldIds; - private Schema eqDeleteRowSchema; - private Schema posDeleteRowSchema; - - Builder(Table table, Schema writeSchema, StructType dsSchema) { - this.table = table; - this.spec = table.spec(); - this.writeSchema = writeSchema; - this.dsSchema = dsSchema; - } - - Builder spec(PartitionSpec newSpec) { - this.spec = newSpec; - return this; - } - - Builder equalityFieldIds(int[] newEqualityFieldIds) { - this.equalityFieldIds = newEqualityFieldIds; - return this; - } - - Builder eqDeleteRowSchema(Schema newEqDeleteRowSchema) { - this.eqDeleteRowSchema = newEqDeleteRowSchema; - return this; - } - - Builder posDelRowSchema(Schema newPosDelRowSchema) { - this.posDeleteRowSchema = newPosDelRowSchema; - return this; - } - - SparkAppenderFactory build() { - Preconditions.checkNotNull(table, "Table must not be null"); - Preconditions.checkNotNull(writeSchema, "Write Schema must not be null"); - Preconditions.checkNotNull(dsSchema, "DS Schema must not be null"); - if (equalityFieldIds != null) { - Preconditions.checkNotNull( - eqDeleteRowSchema, - "Equality Field Ids and Equality Delete Row Schema" + " must be set together"); - } - if (eqDeleteRowSchema != null) { - Preconditions.checkNotNull( - equalityFieldIds, - "Equality Field Ids and Equality Delete Row Schema" + " must be set together"); - } - - return new SparkAppenderFactory( - table.properties(), - writeSchema, - dsSchema, - spec, - equalityFieldIds, - eqDeleteRowSchema, - posDeleteRowSchema); - } - } - - private StructType lazyEqDeleteSparkType() { - if (eqDeleteSparkType == null) { - Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); - this.eqDeleteSparkType = SparkSchemaUtil.convert(eqDeleteRowSchema); - } - return eqDeleteSparkType; - } - - private StructType lazyPosDeleteSparkType() { - if (posDeleteSparkType == null) { - Preconditions.checkNotNull( - posDeleteRowSchema, "Position delete row schema shouldn't be null"); - this.posDeleteSparkType = SparkSchemaUtil.convert(posDeleteRowSchema); - } - return posDeleteSparkType; - } - - @Override - public FileAppender newAppender(OutputFile file, FileFormat fileFormat) { - return newAppender(EncryptionUtil.plainAsEncryptedOutput(file), fileFormat); - } - - @Override - public FileAppender newAppender(EncryptedOutputFile file, FileFormat fileFormat) { - MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties); - try { - switch (fileFormat) { - case PARQUET: - return Parquet.write(file) - .createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dsSchema, msgType)) - .setAll(properties) - .metricsConfig(metricsConfig) - .schema(writeSchema) - .overwrite() - .build(); - - case AVRO: - return Avro.write(file) - .createWriterFunc(ignored -> new SparkAvroWriter(dsSchema)) - .setAll(properties) - .schema(writeSchema) - .overwrite() - .build(); - - case ORC: - return ORC.write(file) - .createWriterFunc(SparkOrcWriter::new) - .setAll(properties) - .metricsConfig(metricsConfig) - .schema(writeSchema) - .overwrite() - .build(); - - default: - throw new UnsupportedOperationException("Cannot write unknown format: " + fileFormat); - } - } catch (IOException e) { - throw new RuntimeIOException(e); - } - } - - @Override - public DataWriter newDataWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - return new DataWriter<>( - newAppender(file, format), - format, - file.encryptingOutputFile().location(), - spec, - partition, - file.keyMetadata()); - } - - @Override - public EqualityDeleteWriter newEqDeleteWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - Preconditions.checkState( - equalityFieldIds != null && equalityFieldIds.length > 0, - "Equality field ids shouldn't be null or empty when creating equality-delete writer"); - Preconditions.checkNotNull( - eqDeleteRowSchema, - "Equality delete row schema shouldn't be null when creating equality-delete writer"); - - try { - switch (format) { - case PARQUET: - return Parquet.writeDeletes(file) - .createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(lazyEqDeleteSparkType(), msgType)) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - case AVRO: - return Avro.writeDeletes(file) - .createWriterFunc(ignored -> new SparkAvroWriter(lazyEqDeleteSparkType())) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - case ORC: - return ORC.writeDeletes(file) - .createWriterFunc(SparkOrcWriter::new) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write equality-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException("Failed to create new equality delete writer", e); - } - } - - @Override - public PositionDeleteWriter newPosDeleteWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - try { - switch (format) { - case PARQUET: - StructType sparkPosDeleteSchema = - SparkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return Parquet.writeDeletes(file) - .createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(sparkPosDeleteSchema, msgType)) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .transformPaths(path -> UTF8String.fromString(path.toString())) - .buildPositionWriter(); - - case AVRO: - return Avro.writeDeletes(file) - .createWriterFunc(ignored -> new SparkAvroWriter(lazyPosDeleteSparkType())) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .buildPositionWriter(); - - case ORC: - return ORC.writeDeletes(file) - .createWriterFunc(SparkOrcWriter::new) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .transformPaths(path -> UTF8String.fromString(path.toString())) - .buildPositionWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write pos-deletes for unsupported file format: " + format); - } - - } catch (IOException e) { - throw new UncheckedIOException("Failed to create new equality delete writer", e); - } - } -} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java deleted file mode 100644 index 0664400c7911..000000000000 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import java.util.List; -import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.TestAppenderFactory; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.unsafe.types.UTF8String; - -public class TestSparkAppenderFactory extends TestAppenderFactory { - - private final StructType sparkType = SparkSchemaUtil.convert(SCHEMA); - - @Override - protected FileAppenderFactory createAppenderFactory( - List equalityFieldIds, Schema eqDeleteSchema, Schema posDeleteRowSchema) { - return SparkAppenderFactory.builderFor(table, table.schema(), sparkType) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .eqDeleteRowSchema(eqDeleteSchema) - .posDelRowSchema(posDeleteRowSchema) - .build(); - } - - @Override - protected InternalRow createRow(Integer id, String data) { - InternalRow row = new GenericInternalRow(2); - row.update(0, id); - row.update(1, UTF8String.fromString(data)); - return row; - } - - @Override - protected StructLikeSet expectedRowSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - for (InternalRow row : rows) { - InternalRowWrapper wrapper = new InternalRowWrapper(sparkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java deleted file mode 100644 index fa1ff9fbfde0..000000000000 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestMergingMetrics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.catalyst.InternalRow; - -public class TestSparkMergingMetrics extends TestMergingMetrics { - - @Override - protected FileAppender writeAndGetAppender(List records) throws IOException { - Table testTable = - new BaseTable(null, "dummy") { - @Override - public Map properties() { - return Collections.emptyMap(); - } - - @Override - public SortOrder sortOrder() { - return SortOrder.unsorted(); - } - - @Override - public PartitionSpec spec() { - return PartitionSpec.unpartitioned(); - } - }; - - File tempFile = File.createTempFile("junit", null, tempDir); - FileAppender appender = - SparkAppenderFactory.builderFor(testTable, SCHEMA, SparkSchemaUtil.convert(SCHEMA)) - .build() - .newAppender(org.apache.iceberg.Files.localOutput(tempFile), fileFormat); - try (FileAppender fileAppender = appender) { - records.stream() - .map(r -> new StructInternalRow(SCHEMA.asStruct()).setStruct(r)) - .forEach(fileAppender::add); - } - return appender; - } -} diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java index 963159fe4ee9..06efb6ba20e4 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java @@ -44,8 +44,6 @@ import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -176,33 +174,6 @@ public void writeUnpartitionedClusteredDataWriter(Blackhole blackhole) throws IO blackhole.consume(writer); } - @Benchmark - @Threads(1) - public void writeUnpartitionedLegacyDataWriter(Blackhole blackhole) throws IOException { - FileIO io = table().io(); - - OutputFileFactory fileFactory = newFileFactory(); - - Schema writeSchema = table().schema(); - StructType sparkWriteType = SparkSchemaUtil.convert(writeSchema); - SparkAppenderFactory appenders = - SparkAppenderFactory.builderFor(table(), writeSchema, sparkWriteType) - .spec(unpartitionedSpec) - .build(); - - TaskWriter writer = - new UnpartitionedWriter<>( - unpartitionedSpec, fileFormat(), appenders, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES); - - try (TaskWriter closableWriter = writer) { - for (InternalRow row : rows) { - closableWriter.write(row); - } - } - - blackhole.consume(writer.complete()); - } - @Benchmark @Threads(1) public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOException { @@ -233,40 +204,6 @@ public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOEx blackhole.consume(writer); } - @Benchmark - @Threads(1) - public void writePartitionedLegacyDataWriter(Blackhole blackhole) throws IOException { - FileIO io = table().io(); - - OutputFileFactory fileFactory = newFileFactory(); - - Schema writeSchema = table().schema(); - StructType sparkWriteType = SparkSchemaUtil.convert(writeSchema); - SparkAppenderFactory appenders = - SparkAppenderFactory.builderFor(table(), writeSchema, sparkWriteType) - .spec(partitionedSpec) - .build(); - - TaskWriter writer = - new SparkPartitionedWriter( - partitionedSpec, - fileFormat(), - appenders, - fileFactory, - io, - TARGET_FILE_SIZE_IN_BYTES, - writeSchema, - sparkWriteType); - - try (TaskWriter closableWriter = writer) { - for (InternalRow row : rows) { - closableWriter.write(row); - } - } - - blackhole.consume(writer.complete()); - } - @Benchmark @Threads(1) public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOException { @@ -297,40 +234,6 @@ public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOExcep blackhole.consume(writer); } - @Benchmark - @Threads(1) - public void writePartitionedLegacyFanoutDataWriter(Blackhole blackhole) throws IOException { - FileIO io = table().io(); - - OutputFileFactory fileFactory = newFileFactory(); - - Schema writeSchema = table().schema(); - StructType sparkWriteType = SparkSchemaUtil.convert(writeSchema); - SparkAppenderFactory appenders = - SparkAppenderFactory.builderFor(table(), writeSchema, sparkWriteType) - .spec(partitionedSpec) - .build(); - - TaskWriter writer = - new SparkPartitionedFanoutWriter( - partitionedSpec, - fileFormat(), - appenders, - fileFactory, - io, - TARGET_FILE_SIZE_IN_BYTES, - writeSchema, - sparkWriteType); - - try (TaskWriter closableWriter = writer) { - for (InternalRow row : rows) { - closableWriter.write(row); - } - } - - blackhole.consume(writer.complete()); - } - @Benchmark @Threads(1) public void writePartitionedClusteredEqualityDeleteWriter(Blackhole blackhole) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java deleted file mode 100644 index c822ed743f85..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ /dev/null @@ -1,328 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Map; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.deletes.EqualityDeleteWriter; -import org.apache.iceberg.deletes.PositionDeleteWriter; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.encryption.EncryptionUtil; -import org.apache.iceberg.exceptions.RuntimeIOException; -import org.apache.iceberg.io.DataWriter; -import org.apache.iceberg.io.DeleteSchemaUtil; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.data.SparkAvroWriter; -import org.apache.iceberg.spark.data.SparkOrcWriter; -import org.apache.iceberg.spark.data.SparkParquetWriters; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.unsafe.types.UTF8String; - -/** - * @deprecated since 1.7.0, will be removed in 1.8.0; use {@link SparkFileWriterFactory} instead. - */ -@Deprecated -class SparkAppenderFactory implements FileAppenderFactory { - private final Map properties; - private final Schema writeSchema; - private final StructType dsSchema; - private final PartitionSpec spec; - private final int[] equalityFieldIds; - private final Schema eqDeleteRowSchema; - private final Schema posDeleteRowSchema; - - private StructType eqDeleteSparkType = null; - private StructType posDeleteSparkType = null; - - SparkAppenderFactory( - Map properties, - Schema writeSchema, - StructType dsSchema, - PartitionSpec spec, - int[] equalityFieldIds, - Schema eqDeleteRowSchema, - Schema posDeleteRowSchema) { - this.properties = properties; - this.writeSchema = writeSchema; - this.dsSchema = dsSchema; - this.spec = spec; - this.equalityFieldIds = equalityFieldIds; - this.eqDeleteRowSchema = eqDeleteRowSchema; - this.posDeleteRowSchema = posDeleteRowSchema; - } - - static Builder builderFor(Table table, Schema writeSchema, StructType dsSchema) { - return new Builder(table, writeSchema, dsSchema); - } - - static class Builder { - private final Table table; - private final Schema writeSchema; - private final StructType dsSchema; - private PartitionSpec spec; - private int[] equalityFieldIds; - private Schema eqDeleteRowSchema; - private Schema posDeleteRowSchema; - - Builder(Table table, Schema writeSchema, StructType dsSchema) { - this.table = table; - this.spec = table.spec(); - this.writeSchema = writeSchema; - this.dsSchema = dsSchema; - } - - Builder spec(PartitionSpec newSpec) { - this.spec = newSpec; - return this; - } - - Builder equalityFieldIds(int[] newEqualityFieldIds) { - this.equalityFieldIds = newEqualityFieldIds; - return this; - } - - Builder eqDeleteRowSchema(Schema newEqDeleteRowSchema) { - this.eqDeleteRowSchema = newEqDeleteRowSchema; - return this; - } - - Builder posDelRowSchema(Schema newPosDelRowSchema) { - this.posDeleteRowSchema = newPosDelRowSchema; - return this; - } - - SparkAppenderFactory build() { - Preconditions.checkNotNull(table, "Table must not be null"); - Preconditions.checkNotNull(writeSchema, "Write Schema must not be null"); - Preconditions.checkNotNull(dsSchema, "DS Schema must not be null"); - if (equalityFieldIds != null) { - Preconditions.checkNotNull( - eqDeleteRowSchema, - "Equality Field Ids and Equality Delete Row Schema" + " must be set together"); - } - if (eqDeleteRowSchema != null) { - Preconditions.checkNotNull( - equalityFieldIds, - "Equality Field Ids and Equality Delete Row Schema" + " must be set together"); - } - - return new SparkAppenderFactory( - table.properties(), - writeSchema, - dsSchema, - spec, - equalityFieldIds, - eqDeleteRowSchema, - posDeleteRowSchema); - } - } - - private StructType lazyEqDeleteSparkType() { - if (eqDeleteSparkType == null) { - Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); - this.eqDeleteSparkType = SparkSchemaUtil.convert(eqDeleteRowSchema); - } - return eqDeleteSparkType; - } - - private StructType lazyPosDeleteSparkType() { - if (posDeleteSparkType == null) { - Preconditions.checkNotNull( - posDeleteRowSchema, "Position delete row schema shouldn't be null"); - this.posDeleteSparkType = SparkSchemaUtil.convert(posDeleteRowSchema); - } - return posDeleteSparkType; - } - - @Override - public FileAppender newAppender(OutputFile file, FileFormat fileFormat) { - return newAppender(EncryptionUtil.plainAsEncryptedOutput(file), fileFormat); - } - - @Override - public FileAppender newAppender(EncryptedOutputFile file, FileFormat fileFormat) { - MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties); - try { - switch (fileFormat) { - case PARQUET: - return Parquet.write(file) - .createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dsSchema, msgType)) - .setAll(properties) - .metricsConfig(metricsConfig) - .schema(writeSchema) - .overwrite() - .build(); - - case AVRO: - return Avro.write(file) - .createWriterFunc(ignored -> new SparkAvroWriter(dsSchema)) - .setAll(properties) - .schema(writeSchema) - .overwrite() - .build(); - - case ORC: - return ORC.write(file) - .createWriterFunc(SparkOrcWriter::new) - .setAll(properties) - .metricsConfig(metricsConfig) - .schema(writeSchema) - .overwrite() - .build(); - - default: - throw new UnsupportedOperationException("Cannot write unknown format: " + fileFormat); - } - } catch (IOException e) { - throw new RuntimeIOException(e); - } - } - - @Override - public DataWriter newDataWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - return new DataWriter<>( - newAppender(file, format), - format, - file.encryptingOutputFile().location(), - spec, - partition, - file.keyMetadata()); - } - - @Override - public EqualityDeleteWriter newEqDeleteWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - Preconditions.checkState( - equalityFieldIds != null && equalityFieldIds.length > 0, - "Equality field ids shouldn't be null or empty when creating equality-delete writer"); - Preconditions.checkNotNull( - eqDeleteRowSchema, - "Equality delete row schema shouldn't be null when creating equality-delete writer"); - - try { - switch (format) { - case PARQUET: - return Parquet.writeDeletes(file) - .createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(lazyEqDeleteSparkType(), msgType)) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - case AVRO: - return Avro.writeDeletes(file) - .createWriterFunc(ignored -> new SparkAvroWriter(lazyEqDeleteSparkType())) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - case ORC: - return ORC.writeDeletes(file) - .createWriterFunc(SparkOrcWriter::new) - .overwrite() - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .equalityFieldIds(equalityFieldIds) - .withKeyMetadata(file.keyMetadata()) - .buildEqualityWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write equality-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException("Failed to create new equality delete writer", e); - } - } - - @Override - public PositionDeleteWriter newPosDeleteWriter( - EncryptedOutputFile file, FileFormat format, StructLike partition) { - try { - switch (format) { - case PARQUET: - StructType sparkPosDeleteSchema = - SparkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return Parquet.writeDeletes(file) - .createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(sparkPosDeleteSchema, msgType)) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .transformPaths(path -> UTF8String.fromString(path.toString())) - .buildPositionWriter(); - - case AVRO: - return Avro.writeDeletes(file) - .createWriterFunc(ignored -> new SparkAvroWriter(lazyPosDeleteSparkType())) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .buildPositionWriter(); - - case ORC: - return ORC.writeDeletes(file) - .createWriterFunc(SparkOrcWriter::new) - .overwrite() - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withPartition(partition) - .withKeyMetadata(file.keyMetadata()) - .transformPaths(path -> UTF8String.fromString(path.toString())) - .buildPositionWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write pos-deletes for unsupported file format: " + format); - } - - } catch (IOException e) { - throw new UncheckedIOException("Failed to create new equality delete writer", e); - } - } -} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java deleted file mode 100644 index 0664400c7911..000000000000 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAppenderFactory.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import java.util.List; -import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.TestAppenderFactory; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.types.StructType; -import org.apache.spark.unsafe.types.UTF8String; - -public class TestSparkAppenderFactory extends TestAppenderFactory { - - private final StructType sparkType = SparkSchemaUtil.convert(SCHEMA); - - @Override - protected FileAppenderFactory createAppenderFactory( - List equalityFieldIds, Schema eqDeleteSchema, Schema posDeleteRowSchema) { - return SparkAppenderFactory.builderFor(table, table.schema(), sparkType) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .eqDeleteRowSchema(eqDeleteSchema) - .posDelRowSchema(posDeleteRowSchema) - .build(); - } - - @Override - protected InternalRow createRow(Integer id, String data) { - InternalRow row = new GenericInternalRow(2); - row.update(0, id); - row.update(1, UTF8String.fromString(data)); - return row; - } - - @Override - protected StructLikeSet expectedRowSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - for (InternalRow row : rows) { - InternalRowWrapper wrapper = new InternalRowWrapper(sparkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java deleted file mode 100644 index 29425398f395..000000000000 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMergingMetrics.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.Files; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestMergingMetrics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.catalyst.InternalRow; - -public class TestSparkMergingMetrics extends TestMergingMetrics { - - @Override - protected FileAppender writeAndGetAppender(List records) throws IOException { - Table testTable = - new BaseTable(null, "dummy") { - @Override - public Map properties() { - return Collections.emptyMap(); - } - - @Override - public SortOrder sortOrder() { - return SortOrder.unsorted(); - } - - @Override - public PartitionSpec spec() { - return PartitionSpec.unpartitioned(); - } - }; - - File tempFile = File.createTempFile("junit", null, tempDir); - FileAppender appender = - SparkAppenderFactory.builderFor(testTable, SCHEMA, SparkSchemaUtil.convert(SCHEMA)) - .build() - .newAppender(Files.localOutput(tempFile), fileFormat); - try (FileAppender fileAppender = appender) { - records.stream() - .map(r -> new StructInternalRow(SCHEMA.asStruct()).setStruct(r)) - .forEach(fileAppender::add); - } - return appender; - } -} From 6c05f35e67093491aa054d0b41b1ada367df1072 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 12 Dec 2024 18:58:56 +0800 Subject: [PATCH 247/313] Core: Log where the missing metadata file is located for Hadoop (#11643) --- .../iceberg/hadoop/HadoopTableOperations.java | 3 ++- .../iceberg/hadoop/TestHadoopCatalog.java | 26 +++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java index 1f3c9182bf5f..1be351ec7370 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java @@ -107,7 +107,8 @@ public TableMetadata refresh() { // no v0 metadata means the table doesn't exist yet return null; } else if (metadataFile == null) { - throw new ValidationException("Metadata file for version %d is missing", ver); + throw new ValidationException( + "Metadata file for version %d is missing under %s", ver, metadataRoot()); } Path nextMetadataFile = getMetadataFile(ver + 1); diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java index 2b342936fdd9..da67bbd4ac38 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java @@ -46,6 +46,7 @@ import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -548,6 +549,31 @@ public void testVersionHintFileMissingMetadata() throws Exception { .hasMessageStartingWith("Table does not exist"); } + @Test + public void testMetadataFileMissing() throws Exception { + addVersionsToTable(table); + + HadoopTableOperations tableOperations = + (HadoopTableOperations) TABLES.newTableOps(tableLocation); + + FileIO io = table.io(); + io.deleteFile(versionHintFile.getPath()); + try (PositionOutputStream stream = io.newOutputFile(versionHintFile.getPath()).create()) { + stream.write("3".getBytes(StandardCharsets.UTF_8)); + } + + // Check the result of the findVersion(), and load the table and check the current snapshotId + assertThat(tableOperations.findVersion()).isEqualTo(3); + assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + .isEqualTo(table.currentSnapshot().snapshotId()); + + io.deleteFile(tableOperations.getMetadataFile(3).toString()); + assertThatThrownBy(() -> TABLES.load(tableLocation)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Metadata file for version 3 is missing under " + new Path(tableLocation, "metadata")); + } + @Test public void testTableName() throws Exception { HadoopCatalog catalog = hadoopCatalog(); From 3053540c5dc7199d84a6a9bfbe0d3e37efe87990 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 12 Dec 2024 18:07:04 +0100 Subject: [PATCH 248/313] Core: Use HEAD request to check if view exists (#11760) --- .../apache/iceberg/rest/CatalogHandlers.java | 6 ++++ .../iceberg/rest/RESTSessionCatalog.java | 12 +++++++ .../iceberg/rest/RESTCatalogAdapter.java | 10 ++++++ .../iceberg/rest/TestRESTViewCatalog.java | 32 +++++++++++++++++++ 4 files changed, 60 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java index 563853e3f033..344308e4caf0 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -507,6 +507,12 @@ private static LoadViewResponse viewResponse(View view) { .build(); } + public static void viewExists(ViewCatalog catalog, TableIdentifier viewIdentifier) { + if (!catalog.viewExists(viewIdentifier)) { + throw new NoSuchViewException("View does not exist: %s", viewIdentifier); + } + } + public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) { View view = catalog.loadView(viewIdentifier); return viewResponse(view); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 5c6fc49984a5..8e8bd2bb70c7 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -1212,6 +1212,18 @@ public List listViews(SessionContext context, Namespace namespa return views.build(); } + @Override + public boolean viewExists(SessionContext context, TableIdentifier identifier) { + checkViewIdentifierIsValid(identifier); + + try { + client.head(paths.view(identifier), headers(context), ErrorHandlers.viewErrorHandler()); + return true; + } catch (NoSuchViewException e) { + return false; + } + } + @Override public View loadView(SessionContext context, TableIdentifier identifier) { Endpoint.check( diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 87b693e206ae..94dd45d4f23d 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -156,6 +156,7 @@ enum Route { CommitTransactionRequest.class, null), LIST_VIEWS(HTTPMethod.GET, ResourcePaths.V1_VIEWS, null, ListTablesResponse.class), + VIEW_EXISTS(HTTPMethod.HEAD, ResourcePaths.V1_VIEW), LOAD_VIEW(HTTPMethod.GET, ResourcePaths.V1_VIEW, null, LoadViewResponse.class), CREATE_VIEW( HTTPMethod.POST, ResourcePaths.V1_VIEWS, CreateViewRequest.class, LoadViewResponse.class), @@ -471,6 +472,15 @@ public T handleRequest( break; } + case VIEW_EXISTS: + { + if (null != asViewCatalog) { + CatalogHandlers.viewExists(asViewCatalog, viewIdentFromPathVars(vars)); + return null; + } + break; + } + case LOAD_VIEW: { if (null != asViewCatalog) { diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java index 85ccdc8f5ddd..40a206896be4 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -50,6 +50,7 @@ import org.eclipse.jetty.servlet.ServletHolder; import org.junit.jupiter.api.AfterEach; 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.ValueSource; @@ -226,6 +227,37 @@ public void testPaginationForListViews(int numberOfItems) { eq(ListTablesResponse.class)); } + @Test + public void viewExistsViaHEADRequest() { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize("test", ImmutableMap.of()); + + catalog.createNamespace(Namespace.of("ns")); + + assertThat(catalog.viewExists(TableIdentifier.of("ns", "view"))).isFalse(); + + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.GET), + eq("v1/config"), + any(), + any(), + eq(ConfigResponse.class), + any(), + any()); + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.HEAD), + eq("v1/namespaces/ns/views/view"), + any(), + any(), + any(), + any(), + any()); + } + @Override protected RESTCatalog catalog() { return restCatalog; From 1e126e24e2c4b639e2509d3f8194e5ceaace6d56 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 12 Dec 2024 18:17:14 +0100 Subject: [PATCH 249/313] Core: Use HEAD request to check if namespace exists (#11761) --- .../apache/iceberg/rest/CatalogHandlers.java | 6 ++++ .../org/apache/iceberg/rest/RESTCatalog.java | 5 ++++ .../iceberg/rest/RESTSessionCatalog.java | 13 +++++++++ .../iceberg/rest/RESTCatalogAdapter.java | 8 +++++ .../apache/iceberg/rest/TestRESTCatalog.java | 29 +++++++++++++++++++ 5 files changed, 61 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java index 344308e4caf0..aeb310854799 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -163,6 +163,12 @@ public static CreateNamespaceResponse createNamespace( .build(); } + public static void namespaceExists(SupportsNamespaces catalog, Namespace namespace) { + if (!catalog.namespaceExists(namespace)) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } + } + public static GetNamespaceResponse loadNamespace( SupportsNamespaces catalog, Namespace namespace) { Map properties = catalog.loadNamespaceMetadata(namespace); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java index 61a7eca272df..73a53de90657 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java @@ -228,6 +228,11 @@ public List listNamespaces(Namespace ns) throws NoSuchNamespaceExcept return nsDelegate.listNamespaces(ns); } + @Override + public boolean namespaceExists(Namespace namespace) { + return nsDelegate.namespaceExists(namespace); + } + @Override public Map loadNamespaceMetadata(Namespace ns) throws NoSuchNamespaceException { return nsDelegate.loadNamespaceMetadata(ns); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 8e8bd2bb70c7..37b70aff3db5 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -652,6 +652,19 @@ public List listNamespaces(SessionContext context, Namespace namespac return namespaces.build(); } + @Override + public boolean namespaceExists(SessionContext context, Namespace namespace) { + checkNamespaceIsValid(namespace); + + try { + client.head( + paths.namespace(namespace), headers(context), ErrorHandlers.namespaceErrorHandler()); + return true; + } catch (NoSuchNamespaceException e) { + return false; + } + } + @Override public Map loadNamespaceMetadata(SessionContext context, Namespace ns) { Endpoint.check(endpoints, Endpoint.V1_LOAD_NAMESPACE); diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 94dd45d4f23d..2fb4defd1224 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -124,6 +124,7 @@ enum Route { ResourcePaths.V1_NAMESPACES, CreateNamespaceRequest.class, CreateNamespaceResponse.class), + NAMESPACE_EXISTS(HTTPMethod.HEAD, ResourcePaths.V1_NAMESPACE), LOAD_NAMESPACE(HTTPMethod.GET, ResourcePaths.V1_NAMESPACE, null, GetNamespaceResponse.class), DROP_NAMESPACE(HTTPMethod.DELETE, ResourcePaths.V1_NAMESPACE), UPDATE_NAMESPACE( @@ -331,6 +332,13 @@ public T handleRequest( } break; + case NAMESPACE_EXISTS: + if (asNamespaceCatalog != null) { + CatalogHandlers.namespaceExists(asNamespaceCatalog, namespaceFromPathVars(vars)); + return null; + } + break; + case LOAD_NAMESPACE: if (asNamespaceCatalog != null) { Namespace namespace = namespaceFromPathVars(vars); diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 973e394b30c7..768d6c3777ee 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -2634,6 +2634,35 @@ public void testNoCleanupForNonCleanableReplaceTransaction() { .isTrue(); } + @Test + public void testNamespaceExistsViaHEADRequest() { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize("test", ImmutableMap.of()); + + assertThat(catalog.namespaceExists(Namespace.of("non-existing"))).isFalse(); + + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.GET), + eq("v1/config"), + any(), + any(), + eq(ConfigResponse.class), + any(), + any()); + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.HEAD), + eq("v1/namespaces/non-existing"), + any(), + any(), + any(), + any(), + any()); + } + private RESTCatalog catalog(RESTCatalogAdapter adapter) { RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); From a3dcfd19fd1b2a709f7bdf013b83836953d49c6f Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Thu, 12 Dec 2024 11:01:44 -0700 Subject: [PATCH 250/313] Hive: Optimize tableExists API in hive catalog (#11597) * Hive: Optimize tableExists API in hive catalog Skip creation of hive table operation when check existence of iceberg table in hive catalog * Add a newline after if/else * Add current thread interrupt * Handle metadata tables and separate the tests * Add comment back * Address feedback * Add extra comment for EcsCatalog override method * Move javadoc around * Added note if hive table with same name exists * Added note if hive table with same name exists * Add test with invalid identifier --- .../apache/iceberg/BaseMetastoreCatalog.java | 2 +- .../org/apache/iceberg/hive/HiveCatalog.java | 37 ++++++++++++++ .../apache/iceberg/hive/HiveTableTest.java | 49 +++++++++++++++++++ 3 files changed, 87 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index e960fe2b63e0..29068df380a9 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -113,7 +113,7 @@ private Table loadMetadataTable(TableIdentifier identifier) { } } - private boolean isValidMetadataIdentifier(TableIdentifier identifier) { + protected boolean isValidMetadataIdentifier(TableIdentifier identifier) { return MetadataTableType.from(identifier.name()) != null && isValidIdentifier(TableIdentifier.of(identifier.namespace().levels())); } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java index 1cf738d736cb..9fd7c6f2eeb0 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java @@ -412,6 +412,43 @@ private void validateTableIsIcebergTableOrView( } } + /** + * Check whether table or metadata table exists. + * + *

    Note: If a hive table with the same identifier exists in catalog, this method will return + * {@code false}. + * + * @param identifier a table identifier + * @return true if the table exists, false otherwise + */ + @Override + public boolean tableExists(TableIdentifier identifier) { + TableIdentifier baseTableIdentifier = identifier; + if (!isValidIdentifier(identifier)) { + if (!isValidMetadataIdentifier(identifier)) { + return false; + } else { + baseTableIdentifier = TableIdentifier.of(identifier.namespace().levels()); + } + } + + String database = baseTableIdentifier.namespace().level(0); + String tableName = baseTableIdentifier.name(); + try { + Table table = clients.run(client -> client.getTable(database, tableName)); + HiveOperationsBase.validateTableIsIceberg(table, fullTableName(name, baseTableIdentifier)); + return true; + } catch (NoSuchTableException | NoSuchObjectException e) { + return false; + } catch (TException e) { + throw new RuntimeException("Failed to check table existence of " + baseTableIdentifier, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException( + "Interrupted in call to check table existence of " + baseTableIdentifier, e); + } + } + @Override public void createNamespace(Namespace namespace, Map meta) { Preconditions.checkArgument( diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java index 5650c4e82458..7f7a56c9a126 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java @@ -388,6 +388,55 @@ public void testHiveTableAndIcebergTableWithSameName(TableType tableType) HIVE_METASTORE_EXTENSION.metastoreClient().dropTable(DB_NAME, hiveTableName); } + @Test + public void testTableExists() throws TException, IOException { + String testTableName = "test_table_exists"; + TableIdentifier identifier = TableIdentifier.of(DB_NAME, testTableName); + TableIdentifier metadataIdentifier = TableIdentifier.of(DB_NAME, testTableName, "partitions"); + TableIdentifier invalidIdentifier = TableIdentifier.of(DB_NAME, "invalid", testTableName); + + assertThat(catalog.tableExists(invalidIdentifier)) + .as("Should return false on invalid identifier") + .isFalse(); + assertThat(catalog.tableExists(identifier)) + .as("Table should not exist before create") + .isFalse(); + catalog.buildTable(identifier, SCHEMA).create(); + + assertThat(catalog.tableExists(identifier)).as("Table should exist after create").isTrue(); + assertThat(catalog.tableExists(metadataIdentifier)) + .as("Metadata table should also exist") + .isTrue(); + + assertThat(catalog.dropTable(identifier)).as("Should drop a table that does exist").isTrue(); + assertThat(catalog.tableExists(identifier)).as("Table should not exist after drop").isFalse(); + assertThat(catalog.tableExists(metadataIdentifier)) + .as("Metadata table should not exist after drop") + .isFalse(); + + HIVE_METASTORE_EXTENSION + .metastoreClient() + .createTable(createHiveTable(testTableName, TableType.EXTERNAL_TABLE)); + assertThat(catalog.tableExists(identifier)) + .as("Should return false when a hive table with the same name exists") + .isFalse(); + assertThat(catalog.tableExists(metadataIdentifier)) + .as("Metadata table should not exist") + .isFalse(); + HIVE_METASTORE_EXTENSION.metastoreClient().dropTable(DB_NAME, testTableName); + + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + assertThat(catalog.tableExists(identifier)) + .as("Should return false if identifier refers to a view") + .isFalse(); + catalog.dropView(identifier); + } + private org.apache.hadoop.hive.metastore.api.Table createHiveTable( String hiveTableName, TableType type) throws IOException { Map parameters = Maps.newHashMap(); From 540d6a6251e31b232fe6ed2413680621454d107a Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Fri, 13 Dec 2024 17:05:11 +0900 Subject: [PATCH 251/313] GCS: Suppress JavaUtilDate in OAuth2RefreshCredentialsHandler (#11773) --- .../apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java | 1 + 1 file changed, 1 insertion(+) diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java index 611e7baaec6e..dfaa502c6005 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java @@ -46,6 +46,7 @@ private OAuth2RefreshCredentialsHandler(Map properties) { this.properties = properties; } + @SuppressWarnings("JavaUtilDate") // GCP API uses java.util.Date @Override public AccessToken refreshAccessToken() { LoadCredentialsResponse response; From c2fd77a3f4c804572c4fbe6081e150510ca9e262 Mon Sep 17 00:00:00 2001 From: abharath9 Date: Sat, 14 Dec 2024 16:49:11 -0600 Subject: [PATCH 252/313] Flink: Add RowConverter for Iceberg Source (#11301) Co-authored-by: Bharath Kumar Avusherla --- .../flink/source/reader/RowConverter.java | 59 +++++ ...TestIcebergSourceBoundedConverterBase.java | 223 ++++++++++++++++++ ...TestIcebergSourceBoundedGenericRecord.java | 192 +++------------ .../source/TestIcebergSourceBoundedRow.java | 52 ++++ 4 files changed, 367 insertions(+), 159 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowConverter.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedRow.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowConverter.java new file mode 100644 index 000000000000..a84384fe17bf --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowConverter.java @@ -0,0 +1,59 @@ +/* + * 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.flink.source.reader; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class RowConverter implements RowDataConverter { + private final DataStructureConverter converter; + private final TypeInformation outputTypeInfo; + + private RowConverter(RowType rowType, TypeInformation rowTypeInfo) { + this.converter = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); + this.outputTypeInfo = rowTypeInfo; + } + + public static RowConverter fromIcebergSchema(org.apache.iceberg.Schema icebergSchema) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); + RowTypeInfo rowTypeInfo = + new RowTypeInfo(tableSchema.getFieldTypes(), tableSchema.getFieldNames()); + return new RowConverter(rowType, rowTypeInfo); + } + + @Override + public Row apply(RowData rowData) { + return (Row) converter.toExternal(rowData); + } + + @Override + public TypeInformation getProducedType() { + return outputTypeInfo; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java new file mode 100644 index 000000000000..5ef387864b90 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedConverterBase.java @@ -0,0 +1,223 @@ +/* + * 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.flink.source; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.reader.ReaderFunction; +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.TypeUtil; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestIcebergSourceBoundedConverterBase { + @TempDir protected Path temporaryFolder; + + @RegisterExtension + static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.ORC, 2, true} + }; + } + + @Parameter(index = 0) + FileFormat fileFormat; + + @Parameter(index = 1) + int parallelism; + + @Parameter(index = 2) + boolean useConverter; + + @TestTemplate + public void testUnpartitionedTable() throws Exception { + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + new GenericAppenderHelper(table, fileFormat, temporaryFolder).appendToTable(expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testPartitionedTable() throws Exception { + String dateStr = "2020-03-20"; + Table table = getPartitionedTable(); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + for (Record expectedRecord : expectedRecords) { + expectedRecord.setField("dt", dateStr); + } + addRecordsToPartitionedTable(table, dateStr, expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testProjection() throws Exception { + Table table = getPartitionedTable(); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + addRecordsToPartitionedTable(table, "2020-03-20", expectedRecords); + // select the "data" field (fieldId == 1) + Schema projectedSchema = TypeUtil.select(TestFixtures.SCHEMA, Sets.newHashSet(1)); + List expectedRows = + Arrays.asList(Row.of(expectedRecords.get(0).get(0)), Row.of(expectedRecords.get(1).get(0))); + TestHelpers.assertRows( + run(projectedSchema, Collections.emptyList(), Collections.emptyMap()), expectedRows); + } + + static Table getPartitionedTable() { + return CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + } + + static TableLoader tableLoader() { + return CATALOG_EXTENSION.tableLoader(); + } + + private void addRecordsToPartitionedTable( + Table table, String dateStr, List expectedRecords) throws IOException { + new GenericAppenderHelper(table, fileFormat, temporaryFolder) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); + } + + private List run() throws Exception { + return run(null, Collections.emptyList(), Collections.emptyMap()); + } + + private List run( + Schema projectedSchema, List filters, Map options) + throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(parallelism); + env.getConfig().enableObjectReuse(); + + Configuration config = new Configuration(); + config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + Table table; + try (TableLoader tableLoader = tableLoader()) { + tableLoader.open(); + table = tableLoader.loadTable(); + } + + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder sourceBuilder = + getSourceBuilder(projectedSchema, filters, readSchema, config, table); + + if (projectedSchema != null) { + sourceBuilder.project(projectedSchema); + } + + sourceBuilder.filters(filters); + sourceBuilder.setAll(options); + + DataStream inputStream = + env.fromSource( + sourceBuilder.build(), + WatermarkStrategy.noWatermarks(), + "testBasicRead", + getTypeInfo(readSchema)); + + DataStream stream = mapToRow(inputStream, readSchema); + + try (CloseableIterator iter = stream.executeAndCollect()) { + return Lists.newArrayList(iter); + } + } + + private IcebergSource.Builder getSourceBuilder( + Schema projectedSchema, + List filters, + Schema readSchema, + Configuration config, + Table table) + throws Exception { + if (useConverter) { + return createSourceBuilderWithConverter(readSchema, config, table); + } + return createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } + + private IcebergSource.Builder createSourceBuilderWithConverter( + Schema readSchema, Configuration config, Table table) throws Exception { + return IcebergSource.forOutputType(getConverter(readSchema, table)) + .tableLoader(tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + private IcebergSource.Builder createSourceBuilderWithReaderFunction( + Table table, Schema projected, List filters, Configuration config) + throws Exception { + return IcebergSource.builder() + .tableLoader(tableLoader()) + .readerFunction(getReaderFunction(projected, table, filters)) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + protected abstract org.apache.iceberg.flink.source.reader.RowDataConverter getConverter( + org.apache.iceberg.Schema icebergSchema, Table table) throws Exception; + + protected ReaderFunction getReaderFunction( + org.apache.iceberg.Schema icebergSchema, Table table, List filters) + throws Exception { + throw new UnsupportedOperationException("No default implementation for getReaderFunction"); + } + + protected abstract TypeInformation getTypeInfo(Schema icebergSchema); + + protected abstract DataStream mapToRow(DataStream inputStream, Schema icebergSchema); +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 4e649d15b1ce..faddce542285 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -18,57 +18,34 @@ */ package org.apache.iceberg.flink.source; -import java.nio.file.Path; -import java.util.Arrays; -import java.util.Collections; import java.util.List; -import java.util.Map; import org.apache.avro.generic.GenericRecord; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.sink.AvroGenericRecordToRowDataMapper; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.flink.source.reader.AvroGenericRecordConverter; import org.apache.iceberg.flink.source.reader.AvroGenericRecordReaderFunction; -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.TypeUtil; -import org.junit.jupiter.api.TestTemplate; +import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; @ExtendWith(ParameterizedTestExtension.class) -public class TestIcebergSourceBoundedGenericRecord { - @TempDir protected Path temporaryFolder; - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); +public class TestIcebergSourceBoundedGenericRecord + extends TestIcebergSourceBoundedConverterBase { @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { @@ -80,143 +57,40 @@ public static Object[][] parameters() { }; } - @Parameter(index = 0) - private FileFormat fileFormat; - - @Parameter(index = 1) - private int parallelism; - - @Parameter(index = 2) - private boolean useConverter; - - @TestTemplate - public void testUnpartitionedTable() throws Exception { - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, temporaryFolder).appendToTable(expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + @Override + protected RowDataConverter getConverter(Schema icebergSchema, Table table) { + return AvroGenericRecordConverter.fromIcebergSchema(icebergSchema, table.name()); } - @TestTemplate - public void testPartitionedTable() throws Exception { - String dateStr = "2020-03-20"; - Table table = - CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - for (int i = 0; i < expectedRecords.size(); ++i) { - expectedRecords.get(i).setField("dt", dateStr); - } - - new GenericAppenderHelper(table, fileFormat, temporaryFolder) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + @Override + protected ReaderFunction getReaderFunction( + Schema icebergSchema, Table table, List filters) throws Exception { + return new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + icebergSchema, + null, + false, + table.io(), + table.encryption(), + filters); } - @TestTemplate - public void testProjection() throws Exception { - Table table = - CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, temporaryFolder) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - // select the "data" field (fieldId == 1) - Schema projectedSchema = TypeUtil.select(TestFixtures.SCHEMA, Sets.newHashSet(1)); - List expectedRows = - Arrays.asList(Row.of(expectedRecords.get(0).get(0)), Row.of(expectedRecords.get(1).get(0))); - TestHelpers.assertRows( - run(projectedSchema, Collections.emptyList(), Collections.emptyMap()), expectedRows); - } - - private List run() throws Exception { - return run(null, Collections.emptyList(), Collections.emptyMap()); - } - - private List run( - Schema projectedSchema, List filters, Map options) - throws Exception { - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(parallelism); - env.getConfig().enableObjectReuse(); - - Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); - Table table; - try (TableLoader tableLoader = CATALOG_EXTENSION.tableLoader()) { - tableLoader.open(); - table = tableLoader.loadTable(); - } - - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); - IcebergSource.Builder sourceBuilder; - if (useConverter) { - sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); - } else { - sourceBuilder = - createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); - } - - if (projectedSchema != null) { - sourceBuilder.project(projectedSchema); - } - - sourceBuilder.filters(filters); - sourceBuilder.setAll(options); - - RowType rowType = FlinkSchemaUtil.convert(readSchema); + @Override + protected TypeInformation getTypeInfo(Schema icebergSchema) { org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); - - DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - new GenericRecordAvroTypeInfo(avroSchema)) - // There are two reasons for converting GenericRecord back to Row. - // 1. Avro GenericRecord/Schema is not serializable. - // 2. leverage the TestHelpers.assertRecords for validation. - .map(AvroGenericRecordToRowDataMapper.forAvroSchema(avroSchema)) - .map(new RowDataToRowMapper(rowType)); - - try (CloseableIterator iter = stream.executeAndCollect()) { - return Lists.newArrayList(iter); - } + AvroSchemaUtil.convert(icebergSchema, TestFixtures.TABLE_IDENTIFIER.name()); + return new GenericRecordAvroTypeInfo(avroSchema); } - private IcebergSource.Builder createSourceBuilderWithReaderFunction( - Table table, Schema projected, List filters, Configuration config) { - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - projected, - null, - false, - table.io(), - table.encryption(), - filters); - - return IcebergSource.builder() - .tableLoader(CATALOG_EXTENSION.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); - } - - private IcebergSource.Builder createSourceBuilderWithConverter( - Table table, Schema readSchema, Configuration config) { - AvroGenericRecordConverter converter = - AvroGenericRecordConverter.fromIcebergSchema(readSchema, table.name()); - return IcebergSource.forOutputType(converter) - .tableLoader(CATALOG_EXTENSION.tableLoader()) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); + @Override + protected DataStream mapToRow(DataStream inputStream, Schema icebergSchema) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, TestFixtures.TABLE_IDENTIFIER.name()); + return inputStream + .map(AvroGenericRecordToRowDataMapper.forAvroSchema(avroSchema)) + .map(new RowDataToRowMapper(rowType)); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedRow.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedRow.java new file mode 100644 index 000000000000..170069fecb0e --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedRow.java @@ -0,0 +1,52 @@ +/* + * 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.flink.source; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.source.reader.RowConverter; +import org.apache.iceberg.flink.source.reader.RowDataConverter; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSourceBoundedRow extends TestIcebergSourceBoundedConverterBase { + + @Override + protected RowDataConverter getConverter(Schema icebergSchema, Table table) { + return RowConverter.fromIcebergSchema(icebergSchema); + } + + @Override + protected TypeInformation getTypeInfo(Schema icebergSchema) { + TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); + return new RowTypeInfo(tableSchema.getFieldTypes(), tableSchema.getFieldNames()); + } + + @Override + protected DataStream mapToRow(DataStream inputStream, Schema icebergSchema) { + return inputStream; + } +} From bcf7b630ef80c63d0836114576154f6fb0990ade Mon Sep 17 00:00:00 2001 From: Liurnly Date: Sun, 15 Dec 2024 15:35:05 +0800 Subject: [PATCH 253/313] Spark 3.5: Fix assertion mismatch in PartitionedWritesTestBase/TestRewritePositionDeleteFilesAction (#11748) --- .../spark/actions/TestRewritePositionDeleteFilesAction.java | 4 ++-- .../apache/iceberg/spark/sql/PartitionedWritesTestBase.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 3bce399b34a9..21719657d187 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -275,7 +275,7 @@ public void testRewriteFilter() throws Exception { .execute(); List newDeleteFiles = except(deleteFiles(table), deleteFiles); - assertThat(newDeleteFiles).as("Should have 4 delete files").hasSize(2); + assertThat(newDeleteFiles).as("Delete files").hasSize(2); List expectedRewrittenFiles = filterFiles(table, deleteFiles, ImmutableList.of(1), ImmutableList.of(2)); @@ -469,7 +469,7 @@ public void testRewriteFilterRemoveDangling() throws Exception { .execute(); List newDeleteFiles = except(deleteFiles(table), deleteFiles); - assertThat(newDeleteFiles).as("Should have 2 new delete files").hasSize(0); + assertThat(newDeleteFiles).as("New delete files").hasSize(0); List expectedRewrittenFiles = filterFiles(table, deleteFiles, ImmutableList.of(0), ImmutableList.of(1)); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java index 97f8e6142dc5..88d18113f19c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java @@ -53,7 +53,7 @@ public void removeTables() { @TestTemplate public void testInsertAppend() { assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) - .as("Should have 5 rows after insert") + .as("Rows before insert") .isEqualTo(3L); sql("INSERT INTO %s VALUES (4, 'd'), (5, 'e')", commitTarget()); @@ -74,7 +74,7 @@ public void testInsertAppend() { @TestTemplate public void testInsertOverwrite() { assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) - .as("Should have 5 rows after insert") + .as("Rows before overwrite") .isEqualTo(3L); // 4 and 5 replace 3 in the partition (id - (id % 3)) = 3 From fd739b32b4713370218cfd8f46ad525bc8c203f1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 15 Dec 2024 15:42:53 +0100 Subject: [PATCH 254/313] Build: Bump nessie from 0.101.0 to 0.101.2 (#11791) Bumps `nessie` from 0.101.0 to 0.101.2. Updates `org.projectnessie.nessie:nessie-client` from 0.101.0 to 0.101.2 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.101.0 to 0.101.2 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.101.0 to 0.101.2 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.101.0 to 0.101.2 --- updated-dependencies: - dependency-name: org.projectnessie.nessie:nessie-client dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-jaxrs-testextension dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-testextension dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 3981ac05ff11..ded11489cf87 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -70,7 +70,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.101.0" +nessie = "0.101.2" netty-buffer = "4.1.115.Final" netty-buffer-compat = "4.1.115.Final" object-client-bundle = "3.3.2" From 592b60408aaf1783f0ca4cdffb9a2b487c392806 Mon Sep 17 00:00:00 2001 From: ajreid21 Date: Sun, 15 Dec 2024 22:52:43 -0800 Subject: [PATCH 255/313] Core: Add missing REST endpoint definitions (#11756) --- .../java/org/apache/iceberg/rest/Endpoint.java | 16 ++++++++++++++++ .../apache/iceberg/rest/RESTSessionCatalog.java | 6 ++++++ .../org/apache/iceberg/rest/ResourcePaths.java | 5 +++++ 3 files changed, 27 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/rest/Endpoint.java b/core/src/main/java/org/apache/iceberg/rest/Endpoint.java index 2a8e6d633297..b4b617b8ec5e 100644 --- a/core/src/main/java/org/apache/iceberg/rest/Endpoint.java +++ b/core/src/main/java/org/apache/iceberg/rest/Endpoint.java @@ -40,6 +40,8 @@ public class Endpoint { Endpoint.create("GET", ResourcePaths.V1_NAMESPACES); public static final Endpoint V1_LOAD_NAMESPACE = Endpoint.create("GET", ResourcePaths.V1_NAMESPACE); + public static final Endpoint V1_NAMESPACE_EXISTS = + Endpoint.create("HEAD", ResourcePaths.V1_NAMESPACE); public static final Endpoint V1_CREATE_NAMESPACE = Endpoint.create("POST", ResourcePaths.V1_NAMESPACES); public static final Endpoint V1_UPDATE_NAMESPACE = @@ -52,6 +54,7 @@ public class Endpoint { // table endpoints public static final Endpoint V1_LIST_TABLES = Endpoint.create("GET", ResourcePaths.V1_TABLES); public static final Endpoint V1_LOAD_TABLE = Endpoint.create("GET", ResourcePaths.V1_TABLE); + public static final Endpoint V1_TABLE_EXISTS = Endpoint.create("HEAD", ResourcePaths.V1_TABLE); public static final Endpoint V1_CREATE_TABLE = Endpoint.create("POST", ResourcePaths.V1_TABLES); public static final Endpoint V1_UPDATE_TABLE = Endpoint.create("POST", ResourcePaths.V1_TABLE); public static final Endpoint V1_DELETE_TABLE = Endpoint.create("DELETE", ResourcePaths.V1_TABLE); @@ -61,10 +64,23 @@ public class Endpoint { Endpoint.create("POST", ResourcePaths.V1_TABLE_REGISTER); public static final Endpoint V1_REPORT_METRICS = Endpoint.create("POST", ResourcePaths.V1_TABLE_METRICS); + public static final Endpoint V1_TABLE_CREDENTIALS = + Endpoint.create("GET", ResourcePaths.V1_TABLE_CREDENTIALS); + + // table scan plan endpoints + public static final Endpoint V1_SUBMIT_TABLE_SCAN_PLAN = + Endpoint.create("POST", ResourcePaths.V1_TABLE_SCAN_PLAN_SUBMIT); + public static final Endpoint V1_FETCH_TABLE_SCAN_PLAN = + Endpoint.create("GET", ResourcePaths.V1_TABLE_SCAN_PLAN); + public static final Endpoint V1_CANCEL_TABLE_SCAN_PLAN = + Endpoint.create("DELETE", ResourcePaths.V1_TABLE_SCAN_PLAN); + public static final Endpoint V1_FETCH_TABLE_SCAN_PLAN_TASKS = + Endpoint.create("POST", ResourcePaths.V1_TABLE_SCAN_PLAN_TASKS); // view endpoints public static final Endpoint V1_LIST_VIEWS = Endpoint.create("GET", ResourcePaths.V1_VIEWS); public static final Endpoint V1_LOAD_VIEW = Endpoint.create("GET", ResourcePaths.V1_VIEW); + public static final Endpoint V1_VIEW_EXISTS = Endpoint.create("HEAD", ResourcePaths.V1_VIEW); public static final Endpoint V1_CREATE_VIEW = Endpoint.create("POST", ResourcePaths.V1_VIEWS); public static final Endpoint V1_UPDATE_VIEW = Endpoint.create("POST", ResourcePaths.V1_VIEW); public static final Endpoint V1_DELETE_VIEW = Endpoint.create("DELETE", ResourcePaths.V1_VIEW); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 37b70aff3db5..903e5dc5182e 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -138,11 +138,13 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog ImmutableSet.builder() .add(Endpoint.V1_LIST_NAMESPACES) .add(Endpoint.V1_LOAD_NAMESPACE) + .add(Endpoint.V1_NAMESPACE_EXISTS) .add(Endpoint.V1_CREATE_NAMESPACE) .add(Endpoint.V1_UPDATE_NAMESPACE) .add(Endpoint.V1_DELETE_NAMESPACE) .add(Endpoint.V1_LIST_TABLES) .add(Endpoint.V1_LOAD_TABLE) + .add(Endpoint.V1_TABLE_EXISTS) .add(Endpoint.V1_CREATE_TABLE) .add(Endpoint.V1_UPDATE_TABLE) .add(Endpoint.V1_DELETE_TABLE) @@ -155,6 +157,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog ImmutableSet.builder() .add(Endpoint.V1_LIST_VIEWS) .add(Endpoint.V1_LOAD_VIEW) + .add(Endpoint.V1_VIEW_EXISTS) .add(Endpoint.V1_CREATE_VIEW) .add(Endpoint.V1_UPDATE_VIEW) .add(Endpoint.V1_DELETE_VIEW) @@ -434,6 +437,7 @@ public void renameTable(SessionContext context, TableIdentifier from, TableIdent @Override public boolean tableExists(SessionContext context, TableIdentifier identifier) { + Endpoint.check(endpoints, Endpoint.V1_TABLE_EXISTS); checkIdentifierIsValid(identifier); try { @@ -654,6 +658,7 @@ public List listNamespaces(SessionContext context, Namespace namespac @Override public boolean namespaceExists(SessionContext context, Namespace namespace) { + Endpoint.check(endpoints, Endpoint.V1_NAMESPACE_EXISTS); checkNamespaceIsValid(namespace); try { @@ -1227,6 +1232,7 @@ public List listViews(SessionContext context, Namespace namespa @Override public boolean viewExists(SessionContext context, TableIdentifier identifier) { + Endpoint.check(endpoints, Endpoint.V1_VIEW_EXISTS); checkViewIdentifierIsValid(identifier); try { diff --git a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java index 5ba7eae28262..d5c11f6052f1 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java +++ b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java @@ -32,10 +32,15 @@ public class ResourcePaths { "/v1/{prefix}/namespaces/{namespace}/properties"; public static final String V1_TABLES = "/v1/{prefix}/namespaces/{namespace}/tables"; public static final String V1_TABLE = "/v1/{prefix}/namespaces/{namespace}/tables/{table}"; + public static final String V1_TABLE_CREDENTIALS = + "/v1/{prefix}/namespaces/{namespace}/tables/{table}/credentials"; public static final String V1_TABLE_REGISTER = "/v1/{prefix}/namespaces/{namespace}/register"; public static final String V1_TABLE_METRICS = "/v1/{prefix}/namespaces/{namespace}/tables/{table}/metrics"; public static final String V1_TABLE_RENAME = "/v1/{prefix}/tables/rename"; + public static final String V1_TABLE_SCAN_PLAN_SUBMIT = "/v1/{prefix}/tables/{table}/plan"; + public static final String V1_TABLE_SCAN_PLAN = "/v1/{prefix}/tables/{table}/plan/{plan-id}"; + public static final String V1_TABLE_SCAN_PLAN_TASKS = "/v1/{prefix}/tables/{table}/tasks"; public static final String V1_TRANSACTIONS_COMMIT = "/v1/{prefix}/transactions/commit"; public static final String V1_VIEWS = "/v1/{prefix}/namespaces/{namespace}/views"; public static final String V1_VIEW = "/v1/{prefix}/namespaces/{namespace}/views/{view}"; From 1851ca1b9e6753aca4834aaeddf93e4c7504a6c9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 16 Dec 2024 08:26:19 +0100 Subject: [PATCH 256/313] Build: Bump software.amazon.awssdk:bom from 2.29.29 to 2.29.34 (#11793) Bumps software.amazon.awssdk:bom from 2.29.29 to 2.29.34. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index ded11489cf87..16bafc7a736a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.29.29" +awssdk-bom = "2.29.34" azuresdk-bom = "1.2.30" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" From 2a5b089aa52b2253318985b007af951909adfade Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 16 Dec 2024 08:50:49 +0100 Subject: [PATCH 257/313] Spark: Read DVs when reading from .position_deletes table (#11657) --- .../iceberg/spark/source/DVIterator.java | 101 ++++++ .../source/PositionDeletesRowReader.java | 5 + .../source/TestPositionDeletesReader.java | 301 ++++++++++++++++++ 3 files changed, 407 insertions(+) create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java new file mode 100644 index 000000000000..7b08b86cbfd0 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.unsafe.types.UTF8String; + +class DVIterator implements CloseableIterator { + private final DeleteFile deleteFile; + private final Schema projection; + private final Map idToConstant; + private final Iterator positions; + private Integer deletedPositionIndex; + private GenericInternalRow row; + + DVIterator( + InputFile inputFile, DeleteFile deleteFile, Schema projection, Map idToConstant) { + this.deleteFile = deleteFile; + this.projection = projection; + this.idToConstant = idToConstant; + List pos = Lists.newArrayList(); + new BaseDeleteLoader(ignored -> inputFile) + .loadPositionDeletes(ImmutableList.of(deleteFile), deleteFile.referencedDataFile()) + .forEach(pos::add); + this.positions = pos.iterator(); + } + + @Override + public boolean hasNext() { + return positions.hasNext(); + } + + @Override + public InternalRow next() { + long position = positions.next(); + + if (null == row) { + List rowValues = Lists.newArrayList(); + for (Types.NestedField column : projection.columns()) { + int fieldId = column.fieldId(); + if (fieldId == MetadataColumns.DELETE_FILE_PATH.fieldId()) { + rowValues.add(UTF8String.fromString(deleteFile.referencedDataFile())); + } else if (fieldId == MetadataColumns.DELETE_FILE_POS.fieldId()) { + rowValues.add(position); + // remember the index where the deleted position needs to be set + deletedPositionIndex = rowValues.size() - 1; + } else if (fieldId == MetadataColumns.PARTITION_COLUMN_ID) { + rowValues.add(idToConstant.get(MetadataColumns.PARTITION_COLUMN_ID)); + } else if (fieldId == MetadataColumns.SPEC_ID_COLUMN_ID) { + rowValues.add(idToConstant.get(MetadataColumns.SPEC_ID_COLUMN_ID)); + } else if (fieldId == MetadataColumns.FILE_PATH_COLUMN_ID) { + rowValues.add(idToConstant.get(MetadataColumns.FILE_PATH_COLUMN_ID)); + } + } + + this.row = new GenericInternalRow(rowValues.toArray()); + } else if (null != deletedPositionIndex) { + // only update the deleted position if necessary, everything else stays the same + row.update(deletedPositionIndex, position); + } + + return row; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Remove is not supported"); + } + + @Override + public void close() {} +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java index 1a894df29166..329bcf085569 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java @@ -33,6 +33,7 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.primitives.Ints; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.rdd.InputFileBlockHolder; import org.apache.spark.sql.catalyst.InternalRow; @@ -90,6 +91,10 @@ protected CloseableIterator open(PositionDeletesScanTask task) { ExpressionUtil.extractByIdInclusive( task.residual(), expectedSchema(), caseSensitive(), Ints.toArray(nonConstantFieldIds)); + if (ContentFileUtil.isDV(task.file())) { + return new DVIterator(inputFile, task.file(), expectedSchema(), idToConstant); + } + return newIterable( inputFile, task.file().format(), diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java new file mode 100644 index 000000000000..5b876dfc57ce --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java @@ -0,0 +1,301 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseScanTaskGroup; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.PositionDeletesTable; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +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.SparkSchemaUtil; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPositionDeletesReader extends TestBase { + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).bucket("data", 16).build(); + + private Table table; + private DataFile dataFile1; + private DataFile dataFile2; + + @TempDir private Path temp; + + @Parameter(index = 0) + private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return ImmutableList.of(2, 3); + } + + @BeforeEach + public void before() throws IOException { + table = + catalog.createTable( + TableIdentifier.of("default", "test"), + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion))); + + GenericRecord record = GenericRecord.create(table.schema()); + List records1 = Lists.newArrayList(); + records1.add(record.copy("id", 29, "data", "a")); + records1.add(record.copy("id", 43, "data", "b")); + records1.add(record.copy("id", 61, "data", "c")); + records1.add(record.copy("id", 89, "data", "d")); + + List records2 = Lists.newArrayList(); + records2.add(record.copy("id", 100, "data", "e")); + records2.add(record.copy("id", 121, "data", "f")); + records2.add(record.copy("id", 122, "data", "g")); + + dataFile1 = writeDataFile(records1); + dataFile2 = writeDataFile(records2); + table.newAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + } + + @AfterEach + public void after() { + catalog.dropTable(TableIdentifier.of("default", "test")); + } + + @TestTemplate + public void readPositionDeletesTableWithNoDeleteFiles() { + Table positionDeletesTable = + catalog.loadTable(TableIdentifier.of("default", "test", "position_deletes")); + + assertThat(positionDeletesTable.newBatchScan().planFiles()).isEmpty(); + } + + @TestTemplate + public void readPositionDeletesTableWithMultipleDeleteFiles() throws IOException { + Pair posDeletes1 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + Lists.newArrayList( + Pair.of(dataFile1.location(), 0L), Pair.of(dataFile1.location(), 1L)), + formatVersion); + + Pair posDeletes2 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + Lists.newArrayList( + Pair.of(dataFile2.location(), 2L), Pair.of(dataFile2.location(), 3L)), + formatVersion); + + DeleteFile deleteFile1 = posDeletes1.first(); + DeleteFile deleteFile2 = posDeletes2.first(); + table + .newRowDelta() + .addDeletes(deleteFile1) + .addDeletes(deleteFile2) + .validateDataFilesExist(posDeletes1.second()) + .validateDataFilesExist(posDeletes2.second()) + .commit(); + + Table positionDeletesTable = + catalog.loadTable(TableIdentifier.of("default", "test", "position_deletes")); + + Schema projectedSchema = + positionDeletesTable + .schema() + .select( + MetadataColumns.DELETE_FILE_PATH.name(), + MetadataColumns.DELETE_FILE_POS.name(), + PositionDeletesTable.DELETE_FILE_PATH); + + List scanTasks = + Lists.newArrayList( + positionDeletesTable.newBatchScan().project(projectedSchema).planFiles()); + assertThat(scanTasks).hasSize(2); + + assertThat(scanTasks.get(0)).isInstanceOf(PositionDeletesScanTask.class); + PositionDeletesScanTask scanTask1 = (PositionDeletesScanTask) scanTasks.get(0); + + try (PositionDeletesRowReader reader = + new PositionDeletesRowReader( + table, + new BaseScanTaskGroup<>(null, ImmutableList.of(scanTask1)), + positionDeletesTable.schema(), + projectedSchema, + false)) { + List actualRows = Lists.newArrayList(); + while (reader.next()) { + actualRows.add(reader.get().copy()); + } + + String dataFileLocation = + formatVersion >= 3 ? deleteFile1.referencedDataFile() : dataFile1.location(); + Object[] first = { + UTF8String.fromString(dataFileLocation), 0L, UTF8String.fromString(deleteFile1.location()) + }; + Object[] second = { + UTF8String.fromString(dataFileLocation), 1L, UTF8String.fromString(deleteFile1.location()) + }; + assertThat(internalRowsToJava(actualRows, projectedSchema)) + .hasSize(2) + .containsExactly(first, second); + } + + assertThat(scanTasks.get(1)).isInstanceOf(PositionDeletesScanTask.class); + PositionDeletesScanTask scanTask2 = (PositionDeletesScanTask) scanTasks.get(1); + try (PositionDeletesRowReader reader = + new PositionDeletesRowReader( + table, + new BaseScanTaskGroup<>(null, ImmutableList.of(scanTask2)), + positionDeletesTable.schema(), + projectedSchema, + false)) { + List actualRows = Lists.newArrayList(); + while (reader.next()) { + actualRows.add(reader.get().copy()); + } + + String dataFileLocation = + formatVersion >= 3 ? deleteFile2.referencedDataFile() : dataFile2.location(); + Object[] first = { + UTF8String.fromString(dataFileLocation), 2L, UTF8String.fromString(deleteFile2.location()) + }; + Object[] second = { + UTF8String.fromString(dataFileLocation), 3L, UTF8String.fromString(deleteFile2.location()) + }; + assertThat(internalRowsToJava(actualRows, projectedSchema)) + .hasSize(2) + .containsExactly(first, second); + } + } + + @TestTemplate + public void readPositionDeletesTableWithDifferentColumnOrdering() throws IOException { + Pair posDeletes1 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + Lists.newArrayList( + Pair.of(dataFile1.location(), 0L), Pair.of(dataFile1.location(), 1L)), + formatVersion); + + DeleteFile deleteFile1 = posDeletes1.first(); + table + .newRowDelta() + .addDeletes(deleteFile1) + .validateDataFilesExist(posDeletes1.second()) + .commit(); + + Table positionDeletesTable = + catalog.loadTable(TableIdentifier.of("default", "test", "position_deletes")); + + // select a few fields in backwards order + Schema projectedSchema = + new Schema(MetadataColumns.DELETE_FILE_POS, MetadataColumns.DELETE_FILE_PATH); + + List scanTasks = + Lists.newArrayList( + positionDeletesTable.newBatchScan().project(projectedSchema).planFiles()); + assertThat(scanTasks).hasSize(1); + + assertThat(scanTasks.get(0)).isInstanceOf(PositionDeletesScanTask.class); + PositionDeletesScanTask scanTask1 = (PositionDeletesScanTask) scanTasks.get(0); + + try (PositionDeletesRowReader reader = + new PositionDeletesRowReader( + table, + new BaseScanTaskGroup<>(null, ImmutableList.of(scanTask1)), + positionDeletesTable.schema(), + projectedSchema, + false)) { + List actualRows = Lists.newArrayList(); + while (reader.next()) { + actualRows.add(reader.get().copy()); + } + + assertThat(internalRowsToJava(actualRows, projectedSchema)) + .hasSize(2) + .containsExactly( + new Object[] {0L, UTF8String.fromString(dataFile1.location())}, + new Object[] {1L, UTF8String.fromString(dataFile1.location())}); + } + } + + private DataFile writeDataFile(List records) throws IOException { + return FileHelpers.writeDataFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + records); + } + + private List internalRowsToJava(List rows, Schema projection) { + return rows.stream().map(row -> toJava(row, projection)).collect(Collectors.toList()); + } + + private Object[] toJava(InternalRow row, Schema projection) { + Object[] values = new Object[row.numFields()]; + for (int i = 0; i < projection.columns().size(); i++) { + values[i] = row.get(i, SparkSchemaUtil.convert(projection.columns().get(i).type())); + } + return values; + } +} From f40ec2096bc078b9fd2b59d6beb32cd77e371ac4 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 16 Dec 2024 11:10:03 +0100 Subject: [PATCH 258/313] Core: Add TableUtil to provide access to a table's format version (#11620) --- .../org/apache/iceberg/SerializableTable.java | 22 ++++- .../java/org/apache/iceberg/TableUtil.java | 41 +++++++++ .../org/apache/iceberg/TestTableUtil.java | 92 +++++++++++++++++++ .../apache/iceberg/catalog/CatalogTests.java | 10 +- .../hadoop/TestTableSerialization.java | 5 + 5 files changed, 163 insertions(+), 7 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/TableUtil.java create mode 100644 core/src/test/java/org/apache/iceberg/TestTableUtil.java diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index a2c0d776423c..b7e2f6da09a4 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -50,6 +50,7 @@ * storage. */ public class SerializableTable implements Table, HasTableOperations, Serializable { + private static final int UNKNOWN_FORMAT_VERSION = -1; private final String name; private final String location; @@ -62,13 +63,14 @@ public class SerializableTable implements Table, HasTableOperations, Serializabl private final FileIO io; private final EncryptionManager encryption; private final Map refs; + private final UUID uuid; + private final int formatVersion; private transient volatile LocationProvider lazyLocationProvider = null; private transient volatile Table lazyTable = null; private transient volatile Schema lazySchema = null; private transient volatile Map lazySpecs = null; private transient volatile SortOrder lazySortOrder = null; - private final UUID uuid; protected SerializableTable(Table table) { this.name = table.name(); @@ -85,6 +87,7 @@ protected SerializableTable(Table table) { this.encryption = table.encryption(); this.refs = SerializableMap.copyOf(table.refs()); this.uuid = table.uuid(); + this.formatVersion = formatVersion(table); } /** @@ -158,6 +161,23 @@ public Map properties() { return properties; } + public int formatVersion() { + if (formatVersion == UNKNOWN_FORMAT_VERSION) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not have a format version"); + } + return formatVersion; + } + + private int formatVersion(Table table) { + if (table instanceof HasTableOperations) { + HasTableOperations ops = (HasTableOperations) table; + return ops.operations().current().formatVersion(); + } else { + return UNKNOWN_FORMAT_VERSION; + } + } + @Override public Schema schema() { if (lazySchema == null) { diff --git a/core/src/main/java/org/apache/iceberg/TableUtil.java b/core/src/main/java/org/apache/iceberg/TableUtil.java new file mode 100644 index 000000000000..c1683dffb189 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/TableUtil.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class TableUtil { + private TableUtil() {} + + /** Returns the format version of the given table */ + public static int formatVersion(Table table) { + Preconditions.checkArgument(null != table, "Invalid table: null"); + + if (table instanceof SerializableTable) { + SerializableTable serializableTable = (SerializableTable) table; + return serializableTable.formatVersion(); + } else if (table instanceof HasTableOperations) { + HasTableOperations ops = (HasTableOperations) table; + return ops.operations().current().formatVersion(); + } else { + throw new IllegalArgumentException( + String.format("%s does not have a format version", table.getClass().getSimpleName())); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestTableUtil.java b/core/src/test/java/org/apache/iceberg/TestTableUtil.java new file mode 100644 index 000000000000..2ccb5c01f3e9 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestTableUtil.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +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.ValueSource; + +public class TestTableUtil { + private static final Namespace NS = Namespace.of("ns"); + private static final TableIdentifier IDENTIFIER = TableIdentifier.of(NS, "test"); + + @TempDir private File tmp; + + private InMemoryCatalog catalog; + + @BeforeEach + public void initCatalog() { + catalog = new InMemoryCatalog(); + catalog.initialize("catalog", ImmutableMap.of()); + catalog.createNamespace(NS); + } + + @Test + public void nullTable() { + assertThatThrownBy(() -> TableUtil.formatVersion(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table: null"); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 3}) + public void formatVersionForBaseTable(int formatVersion) { + Table table = + catalog.createTable( + IDENTIFIER, + new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())), + PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion))); + + assertThat(TableUtil.formatVersion(table)).isEqualTo(formatVersion); + assertThat(TableUtil.formatVersion(SerializableTable.copyOf(table))).isEqualTo(formatVersion); + } + + @Test + public void formatVersionForMetadataTables() { + Table table = + catalog.createTable( + IDENTIFIER, new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get()))); + + for (MetadataTableType type : MetadataTableType.values()) { + Table metadataTable = MetadataTableUtils.createMetadataTableInstance(table, type); + assertThatThrownBy(() -> TableUtil.formatVersion(metadataTable)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "%s does not have a format version", metadataTable.getClass().getSimpleName()); + + assertThatThrownBy(() -> TableUtil.formatVersion(SerializableTable.copyOf(metadataTable))) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + "%s does not have a format version", + SerializableTable.SerializableMetadataTable.class.getName()); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java index 4df91a49033d..ecdf30463472 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -49,6 +49,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableUtil; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.Transaction; import org.apache.iceberg.UpdatePartitionSpec; @@ -1282,9 +1283,7 @@ public void testUpdateTableSpecThenRevert() { .withPartitionSpec(SPEC) .withProperty("format-version", "2") .create(); - assertThat(((BaseTable) table).operations().current().formatVersion()) - .as("Should be a v2 table") - .isEqualTo(2); + assertThat(TableUtil.formatVersion(table)).as("Should be a v2 table").isEqualTo(2); table.updateSpec().addField("id").commit(); @@ -2519,7 +2518,7 @@ public void testConcurrentReplaceTransactionSortOrderConflict() { } @ParameterizedTest - @ValueSource(ints = {1, 2}) + @ValueSource(ints = {1, 2, 3}) public void createTableTransaction(int formatVersion) { if (requiresNamespaceCreate()) { catalog().createNamespace(NS); @@ -2533,8 +2532,7 @@ public void createTableTransaction(int formatVersion) { ImmutableMap.of("format-version", String.valueOf(formatVersion))) .commitTransaction(); - BaseTable table = (BaseTable) catalog().loadTable(TABLE); - assertThat(table.operations().current().formatVersion()).isEqualTo(formatVersion); + assertThat(TableUtil.formatVersion(catalog().loadTable(TABLE))).isEqualTo(formatVersion); } @ParameterizedTest diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java index 5ef4697b4736..d253556f83ea 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java @@ -40,6 +40,7 @@ import org.apache.iceberg.StaticTableOperations; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableUtil; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.Transaction; import org.apache.iceberg.io.CloseableIterable; @@ -67,6 +68,7 @@ public void testSerializableTable() throws IOException, ClassNotFoundException { assertThat(serializableTable).isInstanceOf(HasTableOperations.class); assertThat(((HasTableOperations) serializableTable).operations()) .isInstanceOf(StaticTableOperations.class); + assertThat(TableUtil.formatVersion(serializableTable)).isEqualTo(2); } @Test @@ -106,6 +108,9 @@ public void testSerializableMetadataTable() throws IOException, ClassNotFoundExc assertThatThrownBy(() -> ((HasTableOperations) serializableTable).operations()) .isInstanceOf(UnsupportedOperationException.class) .hasMessageEndingWith("does not support operations()"); + assertThatThrownBy(() -> TableUtil.formatVersion(serializableTable)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageEndingWith("does not have a format version"); } } From 16cc4e95c8260b91b22d29acf808eae9435ba19c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 16 Dec 2024 15:49:36 +0100 Subject: [PATCH 259/313] Build: Bump mkdocs-material from 9.5.47 to 9.5.48 (#11790) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.47 to 9.5.48. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.5.47...9.5.48) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 63985d948b6a..f3b50381d398 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.3.7 -mkdocs-material==9.5.47 +mkdocs-material==9.5.48 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.2 From 791d0fa6cf9408ad3b8a1ef633a059ba1f2255ad Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 16 Dec 2024 19:34:05 +0100 Subject: [PATCH 260/313] Spark 3.4: Add REST catalog to Spark integration tests (#11698) --- spark/v3.4/build.gradle | 22 ++++ .../spark/extensions/TestMetadataTables.java | 2 + .../TestRemoveOrphanFilesProcedure.java | 13 ++- .../apache/iceberg/rest/RESTServerRule.java | 107 ++++++++++++++++++ .../iceberg/spark/SparkCatalogConfig.java | 4 + .../iceberg/spark/SparkCatalogTestBase.java | 10 ++ .../spark/SparkTestBaseWithCatalog.java | 66 +++++++++-- .../actions/TestComputeTableStatsAction.java | 1 + .../iceberg/spark/sql/TestAlterTable.java | 13 ++- .../spark/sql/TestCreateTableAsSelect.java | 2 + .../iceberg/spark/sql/TestNamespaceSQL.java | 14 ++- .../iceberg/spark/sql/TestRefreshTable.java | 8 +- 12 files changed, 242 insertions(+), 20 deletions(-) create mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/rest/RESTServerRule.java diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index a34ea6b92826..32739685581b 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -107,9 +107,14 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') + testImplementation (project(path: ':iceberg-open-api', configuration: 'testFixturesRuntimeElements')) { + transitive = false + } testImplementation libs.sqlite.jdbc testImplementation libs.awaitility testImplementation libs.junit.vintage.engine + // runtime dependencies for running REST Catalog based integration test + testRuntimeOnly libs.jetty.servlet } test { @@ -174,6 +179,12 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') testImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + testImplementation (project(path: ':iceberg-open-api', configuration: 'testFixturesRuntimeElements')) { + transitive = false + } + // runtime dependencies for running REST Catalog based integration test + testRuntimeOnly libs.jetty.servlet + testRuntimeOnly libs.sqlite.jdbc testImplementation libs.avro.avro testImplementation libs.parquet.hadoop @@ -252,6 +263,17 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') integrationImplementation project(path: ":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + + // runtime dependencies for running Hive Catalog based integration test + integrationRuntimeOnly project(':iceberg-hive-metastore') + // runtime dependencies for running REST Catalog based integration test + integrationRuntimeOnly project(path: ':iceberg-core', configuration: 'testArtifacts') + integrationRuntimeOnly (project(path: ':iceberg-open-api', configuration: 'testFixturesRuntimeElements')) { + transitive = false + } + integrationRuntimeOnly libs.jetty.servlet + integrationRuntimeOnly libs.sqlite.jdbc + // Not allowed on our classpath, only the runtime jar is allowed integrationCompileOnly project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}") integrationCompileOnly project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index 21439163848d..37d590f97498 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -528,6 +528,8 @@ public void testFilesTableTimeTravelWithSchemaEvolution() throws Exception { spark.createDataFrame(newRecords, newSparkSchema).coalesce(1).writeTo(tableName).append(); + table.refresh(); + Long currentSnapshotId = table.currentSnapshot().snapshotId(); Dataset actualFilesDs = diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 01e341eead89..2b99eecab3fb 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -460,12 +460,15 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { Table table = Spark3Util.loadIcebergTable(spark, tableName); String statsFileName = "stats-file-" + UUID.randomUUID(); + String location = table.location(); + // not every catalog will return file proto for local directories + // i.e. Hadoop and Hive Catalog do, Jdbc and REST do not + if (!location.startsWith("file:")) { + location = "file:" + location; + } + File statsLocation = - new File(new URI(table.location())) - .toPath() - .resolve("data") - .resolve(statsFileName) - .toFile(); + new File(new URI(location)).toPath().resolve("data").resolve(statsFileName).toFile(); StatisticsFile statisticsFile; try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) { long snapshotId = table.currentSnapshot().snapshotId(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/rest/RESTServerRule.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/rest/RESTServerRule.java new file mode 100644 index 000000000000..f42106e6d89c --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/rest/RESTServerRule.java @@ -0,0 +1,107 @@ +/* + * 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.rest; + +import java.util.Map; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.rules.ExternalResource; + +/** + * This class is to make the {@link RESTCatalogServer} usable for JUnit4 in a similar way to {@link + * RESTServerExtension}. + */ +public class RESTServerRule extends ExternalResource { + public static final String FREE_PORT = "0"; + + private volatile RESTCatalogServer localServer; + private RESTCatalog client; + private final Map config; + + public RESTServerRule() { + config = Maps.newHashMap(); + } + + public RESTServerRule(Map config) { + Map conf = Maps.newHashMap(config); + if (conf.containsKey(RESTCatalogServer.REST_PORT) + && conf.get(RESTCatalogServer.REST_PORT).equals(FREE_PORT)) { + conf.put(RESTCatalogServer.REST_PORT, String.valueOf(RCKUtils.findFreePort())); + } + this.config = conf; + } + + public Map config() { + return config; + } + + public RESTCatalog client() { + if (null == client) { + try { + maybeInitClientAndServer(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + return client; + } + + public String uri() { + return client().properties().get(CatalogProperties.URI); + } + + private void maybeInitClientAndServer() throws Exception { + if (null == localServer) { + synchronized (this) { + if (null == localServer) { + this.localServer = new RESTCatalogServer(config); + this.localServer.start(false); + this.client = RCKUtils.initCatalogClient(config); + } + } + } + } + + @Override + protected void before() throws Throwable { + maybeShutdownClientAndServer(); + maybeInitClientAndServer(); + } + + @Override + protected void after() { + maybeShutdownClientAndServer(); + } + + private void maybeShutdownClientAndServer() { + try { + if (localServer != null) { + localServer.stop(); + localServer = null; + } + if (client != null) { + client.close(); + client = null; + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java index abfd7da0c7bd..e03f500dc46c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java @@ -34,6 +34,10 @@ public enum SparkCatalogConfig { "testhadoop", SparkCatalog.class.getName(), ImmutableMap.of("type", "hadoop", "cache-enabled", "false")), + REST( + "testrest", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "rest", "cache-enabled", "false")), SPARK( "spark_catalog", SparkSessionCatalog.class.getName(), diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogTestBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogTestBase.java index 89323c26100c..6b2b9a1b8082 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogTestBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogTestBase.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark; import java.util.Map; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.Rule; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; @@ -45,6 +47,14 @@ public static Object[][] parameters() { SparkCatalogConfig.SPARK.catalogName(), SparkCatalogConfig.SPARK.implementation(), SparkCatalogConfig.SPARK.properties() + }, + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, REST_SERVER_RULE.uri()) + .build() } }; } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java index d5708c9e575e..1c5d9b711b49 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java @@ -18,6 +18,12 @@ */ package org.apache.iceberg.spark; +import static org.apache.iceberg.CatalogProperties.CATALOG_IMPL; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; + import java.io.File; import java.io.IOException; import java.util.Map; @@ -31,20 +37,42 @@ import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.rest.RESTCatalogServer; +import org.apache.iceberg.rest.RESTServerRule; import org.apache.iceberg.util.PropertyUtil; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.rules.TemporaryFolder; public abstract class SparkTestBaseWithCatalog extends SparkTestBase { protected static File warehouse = null; + @ClassRule + public static final RESTServerRule REST_SERVER_RULE = + new RESTServerRule( + Map.of( + RESTCatalogServer.REST_PORT, + RESTServerRule.FREE_PORT, + // In-memory sqlite database by default is private to the connection that created it. + // If more than 1 jdbc connection backed by in-memory sqlite is created behind one + // JdbcCatalog, then different jdbc connections could provide different views of table + // status even belonging to the same catalog. Reference: + // https://www.sqlite.org/inmemorydb.html + CatalogProperties.CLIENT_POOL_SIZE, + "1")); + + protected static RESTCatalog restCatalog; + @BeforeClass public static void createWarehouse() throws IOException { SparkTestBaseWithCatalog.warehouse = File.createTempFile("warehouse", null); Assert.assertTrue(warehouse.delete()); + restCatalog = REST_SERVER_RULE.client(); } @AfterClass @@ -60,8 +88,8 @@ public static void dropWarehouse() throws IOException { protected final String catalogName; protected final Map catalogConfig; - protected final Catalog validationCatalog; - protected final SupportsNamespaces validationNamespaceCatalog; + protected Catalog validationCatalog; + protected SupportsNamespaces validationNamespaceCatalog; protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table"); protected final String tableName; @@ -77,11 +105,7 @@ public SparkTestBaseWithCatalog( String catalogName, String implementation, Map config) { this.catalogName = catalogName; this.catalogConfig = config; - this.validationCatalog = - catalogName.equals("testhadoop") - ? new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse) - : catalog; - this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + configureValidationCatalog(); spark.conf().set("spark.sql.catalog." + catalogName, implementation); config.forEach( @@ -127,4 +151,32 @@ protected void configurePlanningMode(String table, PlanningMode planningMode) { TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()); } + + private void configureValidationCatalog() { + if (catalogConfig.containsKey(ICEBERG_CATALOG_TYPE)) { + switch (catalogConfig.get(ICEBERG_CATALOG_TYPE)) { + case ICEBERG_CATALOG_TYPE_HADOOP: + this.validationCatalog = + new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse); + break; + case ICEBERG_CATALOG_TYPE_REST: + this.validationCatalog = restCatalog; + break; + case ICEBERG_CATALOG_TYPE_HIVE: + this.validationCatalog = catalog; + break; + default: + throw new IllegalArgumentException("Unknown catalog type"); + } + } else if (catalogConfig.containsKey(CATALOG_IMPL)) { + switch (catalogConfig.get(CATALOG_IMPL)) { + case "org.apache.iceberg.inmemory.InMemoryCatalog": + this.validationCatalog = new InMemoryCatalog(); + break; + default: + throw new IllegalArgumentException("Unknown catalog impl"); + } + } + this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java index 88805a070cb1..26d53832a490 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java @@ -116,6 +116,7 @@ public void testComputeTableStatsAction() throws NoSuchTableException, ParseExce new SimpleRecord(4, "d")); spark.createDataset(records, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); SparkActions actions = SparkActions.get(); + table.refresh(); ComputeTableStats.Result results = actions.computeTableStats(table).columns("id", "data").execute(); assertThat(results).isNotNull(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java index 89857a65e23e..e5c6fcd0aa34 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.spark.sql; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Map; import org.apache.iceberg.catalog.Namespace; @@ -32,7 +35,6 @@ import org.apache.spark.sql.AnalysisException; import org.junit.After; import org.junit.Assert; -import org.junit.Assume; import org.junit.Before; import org.junit.Test; @@ -293,8 +295,13 @@ public void testAlterColumnPositionFirst() { @Test public void testTableRename() { - Assume.assumeFalse( - "Hadoop catalog does not support rename", validationCatalog instanceof HadoopCatalog); + assumeThat(catalogConfig.get(ICEBERG_CATALOG_TYPE)) + .as( + "need to fix https://github.com/apache/iceberg/issues/11154 before enabling this for the REST catalog") + .isNotEqualTo(ICEBERG_CATALOG_TYPE_REST); + assumeThat(validationCatalog) + .as("Hadoop catalog does not support rename") + .isNotInstanceOf(HadoopCatalog.class); Assert.assertTrue("Initial name should exist", validationCatalog.tableExists(tableIdent)); Assert.assertFalse("New name should not exist", validationCatalog.tableExists(renamedIdent)); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java index 13dd8bc85300..ff067291c854 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java @@ -394,6 +394,8 @@ public void testCreateRTASWithPartitionSpecChanging() { + "FROM %s ORDER BY 3, 1", tableName, sourceName); + rtasTable.refresh(); + Schema expectedSchema = new Schema( Types.NestedField.optional(1, "id", Types.LongType.get()), diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java index a0573c5b09b7..f7ad4a6e9113 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java @@ -18,7 +18,10 @@ */ package org.apache.iceberg.spark.sql; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.util.List; @@ -27,6 +30,7 @@ import java.util.stream.Collectors; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.BadRequestException; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -67,6 +71,9 @@ public void testCreateNamespace() { @Test public void testDefaultNamespace() { Assume.assumeFalse("Hadoop has no default namespace configured", isHadoopCatalog); + assumeThat(catalogConfig.get(ICEBERG_CATALOG_TYPE)) + .as("REST has no default namespace configured") + .isNotEqualTo(ICEBERG_CATALOG_TYPE_REST); sql("USE %s", catalogName); @@ -105,8 +112,8 @@ public void testDropNonEmptyNamespace() { "Table should exist", validationCatalog.tableExists(TableIdentifier.of(NS, "table"))); assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) - .isInstanceOf(NamespaceNotEmptyException.class) - .hasMessageStartingWith("Namespace db is not empty."); + .isInstanceOfAny(NamespaceNotEmptyException.class, BadRequestException.class) + .hasMessageContaining("Namespace db is not empty."); sql("DROP TABLE %s.table", fullNamespace); } @@ -141,7 +148,8 @@ public void testListNamespace() { List namespaces = sql("SHOW NAMESPACES IN %s", catalogName); - if (isHadoopCatalog) { + if (isHadoopCatalog + || catalogConfig.get(ICEBERG_CATALOG_TYPE).equals(ICEBERG_CATALOG_TYPE_REST)) { Assert.assertEquals("Should have 1 namespace", 1, namespaces.size()); Set namespaceNames = namespaces.stream().map(arr -> arr[0].toString()).collect(Collectors.toSet()); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java index 7da2dc0882db..ccbad7311afd 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -50,8 +51,11 @@ public void removeTables() { public void testRefreshCommand() { // We are not allowed to change the session catalog after it has been initialized, so build a // new one - if (catalogName.equals(SparkCatalogConfig.SPARK.catalogName()) - || catalogName.equals(SparkCatalogConfig.HADOOP.catalogName())) { + if (Set.of( + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.REST.catalogName()) + .contains(catalogName)) { spark.conf().set("spark.sql.catalog." + catalogName + ".cache-enabled", true); spark = spark.cloneSession(); } From 57ea310475e3336e7ec5b581ec40ca956972822a Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 16 Dec 2024 12:46:05 -0800 Subject: [PATCH 261/313] Parquet: Implement defaults for generic data (#11785) --- .../apache/iceberg/data/DataTestHelpers.java | 17 +- .../iceberg/data/parquet/TestGenericData.java | 256 ++++++++++++++++-- .../data/parquet/BaseParquetReaders.java | 23 +- 3 files changed, 260 insertions(+), 36 deletions(-) diff --git a/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java index 0573897dab72..72e3973382af 100644 --- a/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java +++ b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java @@ -29,14 +29,15 @@ public class DataTestHelpers { private DataTestHelpers() {} public static void assertEquals(Types.StructType struct, Record expected, Record actual) { - List fields = struct.fields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i).type(); - - Object expectedValue = expected.get(i); - Object actualValue = actual.get(i); - - assertEquals(fieldType, expectedValue, actualValue); + Types.StructType expectedType = expected.struct(); + for (Types.NestedField field : struct.fields()) { + Types.NestedField expectedField = expectedType.field(field.fieldId()); + if (expectedField != null) { + assertEquals( + field.type(), expected.getField(expectedField.name()), actual.getField(field.name())); + } else { + assertThat(actual.getField(field.name())).isEqualTo(field.initialDefault()); + } } } diff --git a/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java index 6de56570589c..5c7c11f1d231 100644 --- a/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java +++ b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java @@ -19,7 +19,9 @@ package org.apache.iceberg.data.parquet; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -34,15 +36,14 @@ import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.data.DataTest; import org.apache.iceberg.data.DataTestHelpers; -import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; import org.junit.jupiter.api.Test; @@ -50,14 +51,18 @@ public class TestGenericData extends DataTest { @Override protected void writeAndValidate(Schema schema) throws IOException { - List expected = RandomGenericData.generate(schema, 100, 0L); + writeAndValidate(schema, schema); + } + + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + List expected = RandomGenericData.generate(writeSchema, 100, 12228L); File testFile = File.createTempFile("junit", null, temp.toFile()); assertThat(testFile.delete()).isTrue(); try (FileAppender appender = Parquet.write(Files.localOutput(testFile)) - .schema(schema) + .schema(writeSchema) .createWriterFunc(GenericParquetWriter::buildWriter) .build()) { appender.addAll(expected); @@ -66,29 +71,29 @@ protected void writeAndValidate(Schema schema) throws IOException { List rows; try (CloseableIterable reader = Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) + .project(expectedSchema) + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(expectedSchema, fileSchema)) .build()) { rows = Lists.newArrayList(reader); } for (int i = 0; i < expected.size(); i += 1) { - DataTestHelpers.assertEquals(schema.asStruct(), expected.get(i), rows.get(i)); + DataTestHelpers.assertEquals(expectedSchema.asStruct(), expected.get(i), rows.get(i)); } // test reuseContainers try (CloseableIterable reader = Parquet.read(Files.localInput(testFile)) - .project(schema) + .project(expectedSchema) .reuseContainers() - .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(expectedSchema, fileSchema)) .build()) { - CloseableIterator it = reader.iterator(); - int idx = 0; - while (it.hasNext()) { - GenericRecord actualRecord = (GenericRecord) it.next(); - DataTestHelpers.assertEquals(schema.asStruct(), expected.get(idx), actualRecord); - idx++; + int index = 0; + for (Record actualRecord : reader) { + DataTestHelpers.assertEquals(expectedSchema.asStruct(), expected.get(index), actualRecord); + index += 1; } } } @@ -131,14 +136,225 @@ public void testTwoLevelList() throws IOException { .reuseContainers() .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { - CloseableIterator it = reader.iterator(); - assertThat(it).hasNext(); - while (it.hasNext()) { - GenericRecord actualRecord = (GenericRecord) it.next(); + for (Record actualRecord : reader) { assertThat(actualRecord.get(0, ArrayList.class)).first().isEqualTo(expectedBinary); assertThat(actualRecord.get(1, ByteBuffer.class)).isEqualTo(expectedBinary); - assertThat(it).isExhausted(); } + + assertThat(Lists.newArrayList(reader).size()).isEqualTo(1); } } + + @Test + public void testMissingRequiredWithoutDefault() { + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withDoc("Missing required field with no default") + .build()); + + assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Missing required field: missing_str"); + } + + @Test + public void testDefaultValues() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withInitialDefault("orange") + .build(), + NestedField.optional("missing_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNullDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + NestedField.optional("missing_date").withId(3).ofType(Types.DateType.get()).build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNestedDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + NestedField.optional("nested") + .withId(3) + .ofType(Types.StructType.of(required(4, "inner", Types.StringType.get()))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + NestedField.optional("nested") + .withId(3) + .ofType( + Types.StructType.of( + required(4, "inner", Types.StringType.get()), + NestedField.optional("missing_inner_float") + .withId(5) + .ofType(Types.FloatType.get()) + .withInitialDefault(-0.0F) + .build())) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testMapNestedDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of(required(6, "value_str", Types.StringType.get())))) + .withDoc("Used to test nested map value field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of( + required(6, "value_str", Types.StringType.get()), + Types.NestedField.optional("value_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testListNestedDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, Types.StructType.of(required(5, "element_str", Types.StringType.get())))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, + Types.StructType.of( + required(5, "element_str", Types.StringType.get()), + Types.NestedField.optional("element_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } } diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java index 38fd69393023..b3a1948d0a35 100644 --- a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java @@ -263,6 +263,7 @@ public ParquetValueReader struct( int defaultMaxDefinitionLevel = type.getMaxDefinitionLevel(currentPath()); for (Types.NestedField field : expectedFields) { int id = field.fieldId(); + ParquetValueReader reader = readersById.get(id); if (idToConstant.containsKey(id)) { // containsKey is used because the constant may be null int fieldMaxDefinitionLevel = @@ -276,15 +277,21 @@ public ParquetValueReader struct( } else if (id == MetadataColumns.IS_DELETED.fieldId()) { reorderedFields.add(ParquetValueReaders.constant(false)); types.add(null); + } else if (reader != null) { + reorderedFields.add(reader); + types.add(typesById.get(id)); + } else if (field.initialDefault() != null) { + reorderedFields.add( + ParquetValueReaders.constant( + field.initialDefault(), + maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel))); + types.add(typesById.get(id)); + } else if (field.isOptional()) { + reorderedFields.add(ParquetValueReaders.nulls()); + types.add(null); } else { - ParquetValueReader reader = readersById.get(id); - if (reader != null) { - reorderedFields.add(reader); - types.add(typesById.get(id)); - } else { - reorderedFields.add(ParquetValueReaders.nulls()); - types.add(null); - } + throw new IllegalArgumentException( + String.format("Missing required field: %s", field.name())); } } From b9b61b1d72ebb192d5e90453ff7030ece73d2603 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 16 Dec 2024 14:31:01 -0800 Subject: [PATCH 262/313] Avro: Support default values for generic data (#11786) --- .../apache/iceberg/data/avro/DataReader.java | 4 + .../iceberg/data/avro/GenericReaders.java | 34 +++ .../iceberg/data/avro/IcebergDecoder.java | 4 +- .../iceberg/data/avro/PlannedDataReader.java | 187 ++++++++++++++ .../apache/iceberg/data/avro/RawDecoder.java | 32 +++ .../encryption/KeyMetadataDecoder.java | 2 +- .../iceberg/avro/TestAvroDataWriter.java | 4 +- .../iceberg/avro/TestAvroDeleteWriters.java | 11 +- .../iceberg/avro/TestAvroFileSplit.java | 4 +- .../avro/TestEncryptedAvroFileSplit.java | 4 +- .../apache/iceberg/data/BaseDeleteLoader.java | 4 +- .../apache/iceberg/data/GenericReader.java | 5 +- .../iceberg/data/avro/TestGenericData.java | 234 +++++++++++++++++- .../data/avro/TestGenericReadProjection.java | 2 +- .../iceberg/io/TestAppenderFactory.java | 7 +- .../iceberg/io/TestFileWriterFactory.java | 7 +- .../io/TestGenericSortedPosDeleteWriter.java | 7 +- .../io/TestTaskEqualityDeltaWriter.java | 7 +- .../AbstractTestFlinkAvroReaderWriter.java | 4 +- .../mr/mapreduce/IcebergInputFormat.java | 12 +- 20 files changed, 533 insertions(+), 42 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/data/avro/PlannedDataReader.java diff --git a/core/src/main/java/org/apache/iceberg/data/avro/DataReader.java b/core/src/main/java/org/apache/iceberg/data/avro/DataReader.java index 1cc901d15bc1..dbb1df055035 100644 --- a/core/src/main/java/org/apache/iceberg/data/avro/DataReader.java +++ b/core/src/main/java/org/apache/iceberg/data/avro/DataReader.java @@ -36,6 +36,10 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +/** + * @deprecated will be removed in 2.0.0; use {@link PlannedDataReader} instead. + */ +@Deprecated public class DataReader implements DatumReader, SupportsRowPosition { public static DataReader create( diff --git a/core/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java b/core/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java index 91a728d53d38..b07ab5d18681 100644 --- a/core/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java +++ b/core/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java @@ -32,6 +32,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.Pair; class GenericReaders { private GenericReaders() {} @@ -52,6 +53,11 @@ static ValueReader timestamptz() { return TimestamptzReader.INSTANCE; } + static ValueReader struct( + List>> readPlan, StructType struct) { + return new PlannedRecordReader(readPlan, struct); + } + static ValueReader struct( StructType struct, List> readers, Map idToConstant) { return new GenericRecordReader(readers, struct, idToConstant); @@ -101,6 +107,34 @@ public OffsetDateTime read(Decoder decoder, Object reuse) throws IOException { } } + private static class PlannedRecordReader extends ValueReaders.PlannedStructReader { + private final StructType structType; + + private PlannedRecordReader(List>> readPlan, StructType struct) { + super(readPlan); + this.structType = struct; + } + + @Override + protected Record reuseOrCreate(Object reuse) { + if (reuse instanceof Record) { + return (Record) reuse; + } else { + return GenericRecord.create(structType); + } + } + + @Override + protected Object get(Record struct, int pos) { + return struct.get(pos); + } + + @Override + protected void set(Record struct, int pos, Object value) { + struct.set(pos, value); + } + } + private static class GenericRecordReader extends ValueReaders.StructReader { private final StructType structType; diff --git a/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java b/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java index 89513b7e0bed..f21bae037103 100644 --- a/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java +++ b/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java @@ -103,9 +103,7 @@ public void addSchema(org.apache.iceberg.Schema writeSchema) { private void addSchema(Schema writeSchema) { long fp = SchemaNormalization.parsingFingerprint64(writeSchema); - RawDecoder decoder = - new RawDecoder<>( - readSchema, avroSchema -> DataReader.create(readSchema, avroSchema), writeSchema); + RawDecoder decoder = RawDecoder.create(readSchema, PlannedDataReader::create, writeSchema); decoders.put(fp, decoder); } diff --git a/core/src/main/java/org/apache/iceberg/data/avro/PlannedDataReader.java b/core/src/main/java/org/apache/iceberg/data/avro/PlannedDataReader.java new file mode 100644 index 000000000000..c7ec2e6091cc --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/data/avro/PlannedDataReader.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.data.avro; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.AvroWithPartnerVisitor; +import org.apache.iceberg.avro.SupportsRowPosition; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; + +public class PlannedDataReader implements DatumReader, SupportsRowPosition { + + public static PlannedDataReader create(org.apache.iceberg.Schema expectedSchema) { + return create(expectedSchema, ImmutableMap.of()); + } + + public static PlannedDataReader create( + org.apache.iceberg.Schema expectedSchema, Map idToConstant) { + return new PlannedDataReader<>(expectedSchema, idToConstant); + } + + private final org.apache.iceberg.Schema expectedSchema; + private final Map idToConstant; + private ValueReader reader; + + protected PlannedDataReader( + org.apache.iceberg.Schema expectedSchema, Map idToConstant) { + this.expectedSchema = expectedSchema; + this.idToConstant = idToConstant; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema fileSchema) { + this.reader = + (ValueReader) + AvroWithPartnerVisitor.visit( + expectedSchema.asStruct(), + fileSchema, + new ReadBuilder(idToConstant), + AvroWithPartnerVisitor.FieldIDAccessors.get()); + } + + @Override + public T read(T reuse, Decoder decoder) throws IOException { + return reader.read(decoder, reuse); + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + private static class ReadBuilder extends AvroWithPartnerVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public ValueReader record(Type partner, Schema record, List> fieldReaders) { + if (partner == null) { + return ValueReaders.skipStruct(fieldReaders); + } + + Types.StructType expected = partner.asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldReaders, idToConstant); + + return GenericReaders.struct(readPlan, expected); + } + + @Override + public ValueReader union(Type partner, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader array(Type ignored, Schema array, ValueReader elementReader) { + return ValueReaders.array(elementReader); + } + + @Override + public ValueReader arrayMap( + Type ignored, Schema map, ValueReader keyReader, ValueReader valueReader) { + return ValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader map(Type ignored, Schema map, ValueReader valueReader) { + return ValueReaders.map(ValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Type partner, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + return GenericReaders.dates(); + + case "time-micros": + return GenericReaders.times(); + + case "timestamp-micros": + if (AvroSchemaUtil.isTimestamptz(primitive)) { + return GenericReaders.timestamptz(); + } + return GenericReaders.timestamps(); + + case "decimal": + return ValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + ((LogicalTypes.Decimal) logicalType).getScale()); + + case "uuid": + return ValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + if (partner != null && partner.typeId() == Type.TypeID.LONG) { + return ValueReaders.intsAsLongs(); + } + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + if (partner != null && partner.typeId() == Type.TypeID.DOUBLE) { + return ValueReaders.floatsAsDoubles(); + } + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + // might want to use a binary-backed container like Utf8 + return ValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive.getFixedSize()); + case BYTES: + return ValueReaders.byteBuffers(); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java b/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java index c32ea707bfab..436cba05c73a 100644 --- a/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java +++ b/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java @@ -33,6 +33,27 @@ public class RawDecoder extends MessageDecoder.BaseDecoder { private static final ThreadLocal DECODER = new ThreadLocal<>(); + /** + * Creates a new {@link MessageDecoder} that constructs datum instances described by the {@link + * Schema readSchema}. + * + *

    The {@code readSchema} is used for the expected schema and the {@code writeSchema} is the + * schema used to decode buffers. The {@code writeSchema} must be the schema that was used to + * encode all buffers decoded by this class. + * + * @param readSchema an Iceberg schema to produce when reading + * @param readerFunction a function that produces a DatumReader from the read schema + * @param writeSchema an Avro schema that describes serialized data to be read + */ + public static RawDecoder create( + org.apache.iceberg.Schema readSchema, + Function> readerFunction, + Schema writeSchema) { + DatumReader reader = readerFunction.apply(readSchema); + reader.setSchema(writeSchema); + return new RawDecoder<>(reader); + } + private final DatumReader reader; /** @@ -42,7 +63,11 @@ public class RawDecoder extends MessageDecoder.BaseDecoder { *

    The {@code readSchema} is used for the expected schema and the {@code writeSchema} is the * schema used to decode buffers. The {@code writeSchema} must be the schema that was used to * encode all buffers decoded by this class. + * + * @deprecated will be removed in 2.0.0; use {@link #create(org.apache.iceberg.Schema, Function, + * Schema)} instead */ + @Deprecated public RawDecoder( org.apache.iceberg.Schema readSchema, Function> readerFunction, @@ -51,6 +76,13 @@ public RawDecoder( this.reader.setSchema(writeSchema); } + /** + * Creates a new {@link MessageDecoder} that constructs datum instances using the {@code reader}. + */ + private RawDecoder(DatumReader reader) { + this.reader = reader; + } + @Override public D decode(InputStream stream, D reuse) { BinaryDecoder decoder = DecoderFactory.get().directBinaryDecoder(stream, DECODER.get()); diff --git a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java index 7e57163d73ea..a09951728173 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java +++ b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java @@ -66,7 +66,7 @@ public StandardKeyMetadata decode(InputStream stream, StandardKeyMetadata reuse) RawDecoder decoder = decoders.get(writeSchemaVersion); if (decoder == null) { - decoder = new RawDecoder<>(readSchema, GenericAvroReader::create, writeSchema); + decoder = RawDecoder.create(readSchema, GenericAvroReader::create, writeSchema); decoders.put(writeSchemaVersion, decoder); } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroDataWriter.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroDataWriter.java index 62f736e2c517..a9b9e7a6a74b 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroDataWriter.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroDataWriter.java @@ -32,7 +32,7 @@ import org.apache.iceberg.SortOrder; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -103,7 +103,7 @@ public void testDataWriter() throws IOException { try (AvroIterable reader = Avro.read(file.toInputFile()) .project(SCHEMA) - .createReaderFunc(DataReader::create) + .createResolvingReader(PlannedDataReader::create) .build()) { writtenRecords = Lists.newArrayList(reader); } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java index 504ef7aad5b3..86bb74c5a397 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java @@ -33,8 +33,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteWriter; @@ -102,7 +102,10 @@ public void testEqualityDeleteWriter() throws IOException { List deletedRecords; try (AvroIterable reader = - Avro.read(out.toInputFile()).project(SCHEMA).createReaderFunc(DataReader::create).build()) { + Avro.read(out.toInputFile()) + .project(SCHEMA) + .createResolvingReader(PlannedDataReader::create) + .build()) { deletedRecords = Lists.newArrayList(reader); } @@ -158,7 +161,7 @@ public void testPositionDeleteWriter() throws IOException { try (AvroIterable reader = Avro.read(out.toInputFile()) .project(deleteSchema) - .createReaderFunc(DataReader::create) + .createResolvingReader(PlannedDataReader::create) .build()) { deletedRecords = Lists.newArrayList(reader); } @@ -212,7 +215,7 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { try (AvroIterable reader = Avro.read(out.toInputFile()) .project(deleteSchema) - .createReaderFunc(DataReader::create) + .createResolvingReader(PlannedDataReader::create) .build()) { deletedRecords = Lists.newArrayList(reader); } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroFileSplit.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroFileSplit.java index 12b1326fc1be..fac537666854 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroFileSplit.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroFileSplit.java @@ -30,8 +30,8 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; @@ -186,7 +186,7 @@ public List readAvro(InputFile in, Schema projection, long start, long l throws IOException { try (AvroIterable reader = Avro.read(in) - .createReaderFunc(DataReader::create) + .createResolvingReader(PlannedDataReader::create) .split(start, length) .project(projection) .build()) { diff --git a/core/src/test/java/org/apache/iceberg/avro/TestEncryptedAvroFileSplit.java b/core/src/test/java/org/apache/iceberg/avro/TestEncryptedAvroFileSplit.java index 9020a1230271..efb5de3e96a7 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestEncryptedAvroFileSplit.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestEncryptedAvroFileSplit.java @@ -30,8 +30,8 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.encryption.EncryptedFiles; import org.apache.iceberg.encryption.EncryptedInputFile; import org.apache.iceberg.encryption.EncryptedOutputFile; @@ -199,7 +199,7 @@ public List readAvro(InputFile in, Schema projection, long start, long l throws IOException { try (AvroIterable reader = Avro.read(in) - .createReaderFunc(DataReader::create) + .createResolvingReader(PlannedDataReader::create) .split(start, length) .project(projection) .build()) { diff --git a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java index 1b7a92f0682b..d0c50a614620 100644 --- a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java +++ b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java @@ -31,7 +31,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.data.orc.GenericOrcReader; import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.deletes.Deletes; @@ -235,7 +235,7 @@ private CloseableIterable openDeletes( return Avro.read(inputFile) .project(projection) .reuseContainers() - .createReaderFunc(DataReader::create) + .createResolvingReader(PlannedDataReader::create) .build(); case PARQUET: diff --git a/data/src/main/java/org/apache/iceberg/data/GenericReader.java b/data/src/main/java/org/apache/iceberg/data/GenericReader.java index 590b01b228ed..aaf4b76ca851 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericReader.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericReader.java @@ -26,7 +26,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.TableScan; import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.data.orc.GenericOrcReader; import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.expressions.Evaluator; @@ -101,8 +101,7 @@ private CloseableIterable openFile(FileScanTask task, Schema fileProject Avro.ReadBuilder avro = Avro.read(input) .project(fileProjection) - .createReaderFunc( - avroSchema -> DataReader.create(fileProjection, avroSchema, partition)) + .createResolvingReader(schema -> PlannedDataReader.create(schema, partition)) .split(task.start(), task.length()); if (reuseContainers) { diff --git a/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java index 83e8c09449e4..651df22cfc15 100644 --- a/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java +++ b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java @@ -18,7 +18,9 @@ */ package org.apache.iceberg.data.avro; +import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -33,18 +35,24 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; public class TestGenericData extends DataTest { @Override protected void writeAndValidate(Schema schema) throws IOException { - List expected = RandomGenericData.generate(schema, 100, 0L); + writeAndValidate(schema, schema); + } + + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + List expected = RandomGenericData.generate(writeSchema, 100, 0L); File testFile = File.createTempFile("junit", null, temp.toFile()); assertThat(testFile.delete()).isTrue(); try (FileAppender writer = Avro.write(Files.localOutput(testFile)) - .schema(schema) + .schema(writeSchema) .createWriterFunc(DataWriter::create) .named("test") .build()) { @@ -56,14 +64,230 @@ protected void writeAndValidate(Schema schema) throws IOException { List rows; try (AvroIterable reader = Avro.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(DataReader::create) + .project(expectedSchema) + .createResolvingReader(PlannedDataReader::create) .build()) { rows = Lists.newArrayList(reader); } for (int i = 0; i < expected.size(); i += 1) { - DataTestHelpers.assertEquals(schema.asStruct(), expected.get(i), rows.get(i)); + DataTestHelpers.assertEquals(expectedSchema.asStruct(), expected.get(i), rows.get(i)); } } + + @Test + public void testMissingRequiredWithoutDefault() { + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withDoc("Missing required field with no default") + .build()); + + assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Missing required field: missing_str"); + } + + @Test + public void testDefaultValues() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withInitialDefault("orange") + .build(), + Types.NestedField.optional("missing_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNullDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("missing_date") + .withId(3) + .ofType(Types.DateType.get()) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNestedDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType(Types.StructType.of(required(4, "inner", Types.StringType.get()))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType( + Types.StructType.of( + required(4, "inner", Types.StringType.get()), + Types.NestedField.optional("missing_inner_float") + .withId(5) + .ofType(Types.FloatType.get()) + .withInitialDefault(-0.0F) + .build())) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testMapNestedDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of(required(6, "value_str", Types.StringType.get())))) + .withDoc("Used to test nested map value field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of( + required(6, "value_str", Types.StringType.get()), + Types.NestedField.optional("value_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testListNestedDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, Types.StructType.of(required(5, "element_str", Types.StringType.get())))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, + Types.StructType.of( + required(5, "element_str", Types.StringType.get()), + Types.NestedField.optional("element_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } } diff --git a/data/src/test/java/org/apache/iceberg/data/avro/TestGenericReadProjection.java b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericReadProjection.java index b6083906c74b..776dd1466ab7 100644 --- a/data/src/test/java/org/apache/iceberg/data/avro/TestGenericReadProjection.java +++ b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericReadProjection.java @@ -48,7 +48,7 @@ protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema Iterable records = Avro.read(Files.localInput(file)) .project(readSchema) - .createReaderFunc(DataReader::create) + .createResolvingReader(PlannedDataReader::create) .build(); return Iterables.getOnlyElement(records); diff --git a/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java b/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java index 83f1bf261063..4d3e6a8ac9c6 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java @@ -38,7 +38,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.data.orc.GenericOrcReader; import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.deletes.EqualityDeleteWriter; @@ -337,7 +337,10 @@ private CloseableIterable createReader(Schema schema, InputFile inputFil .build(); case AVRO: - return Avro.read(inputFile).project(schema).createReaderFunc(DataReader::create).build(); + return Avro.read(inputFile) + .project(schema) + .createResolvingReader(PlannedDataReader::create) + .build(); case ORC: return ORC.read(inputFile) diff --git a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java index 0acb173f0923..ab1d295125f2 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java @@ -41,7 +41,7 @@ import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.data.orc.GenericOrcReader; import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.deletes.EqualityDeleteWriter; @@ -476,7 +476,10 @@ private List readFile(Schema schema, InputFile inputFile) throws IOExcep case AVRO: try (CloseableIterable records = - Avro.read(inputFile).project(schema).createReaderFunc(DataReader::create).build()) { + Avro.read(inputFile) + .project(schema) + .createResolvingReader(PlannedDataReader::create) + .build()) { return ImmutableList.copyOf(records); } diff --git a/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java b/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java index 629df03e37cd..e7f9d90f0bb2 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java @@ -40,7 +40,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.data.orc.GenericOrcReader; import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.encryption.EncryptedOutputFile; @@ -314,7 +314,10 @@ private List readRecordsAsList(Schema schema, CharSequence path) throws case AVRO: iterable = - Avro.read(inputFile).project(schema).createReaderFunc(DataReader::create).build(); + Avro.read(inputFile) + .project(schema) + .createResolvingReader(PlannedDataReader::create) + .build(); break; case ORC: diff --git a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java index b1688e6653f2..71c112918e38 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java @@ -45,7 +45,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.data.orc.GenericOrcReader; import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.deletes.DeleteGranularity; @@ -610,7 +610,10 @@ private List readRecordsAsList(Schema schema, CharSequence path) throws case AVRO: iterable = - Avro.read(inputFile).project(schema).createReaderFunc(DataReader::create).build(); + Avro.read(inputFile) + .project(schema) + .createResolvingReader(PlannedDataReader::create) + .build(); break; case ORC: diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java index cbf49ae6faa9..4a59dcfd1e09 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/AbstractTestFlinkAvroReaderWriter.java @@ -36,8 +36,8 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.avro.DataReader; import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.io.CloseableIterable; @@ -116,7 +116,7 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n try (CloseableIterable reader = Avro.read(Files.localInput(rowDataFile)) .project(schema) - .createReaderFunc(DataReader::create) + .createResolvingReader(PlannedDataReader::create) .build()) { Iterator expected = expectedRows.iterator(); Iterator records = reader.iterator(); diff --git a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index 492729d97338..bb39644e42c1 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -55,7 +55,7 @@ import org.apache.iceberg.data.GenericDeleteFilter; import org.apache.iceberg.data.IdentityPartitionConverters; import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.PlannedDataReader; import org.apache.iceberg.data.orc.GenericOrcReader; import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.encryption.EncryptedFiles; @@ -389,12 +389,10 @@ private CloseableIterable newAvroIterable( throw new UnsupportedOperationException( "Avro support not yet supported for Pig and Hive"); case GENERIC: - avroReadBuilder.createReaderFunc( - (expIcebergSchema, expAvroSchema) -> - DataReader.create( - expIcebergSchema, - expAvroSchema, - constantsMap(task, IdentityPartitionConverters::convertConstant))); + avroReadBuilder.createResolvingReader( + schema -> + PlannedDataReader.create( + schema, constantsMap(task, IdentityPartitionConverters::convertConstant))); } return applyResidualFiltering(avroReadBuilder.build(), task.residual(), readSchema); } From ac865e334e143dfd9e33011d8cf710b46d91f1e5 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Tue, 17 Dec 2024 13:52:36 +0530 Subject: [PATCH 263/313] REST: Use `apache/iceberg-rest-fixture` docker image (#11673) * REST: Use apache/iceberg-rest-fixture docker image * Fix CI failure --- kafka-connect/kafka-connect-runtime/docker/docker-compose.yml | 3 ++- site/docs/spark-quickstart.md | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml b/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml index 202180289d96..0b44e5c841f9 100644 --- a/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml +++ b/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml @@ -41,7 +41,7 @@ services: entrypoint: mc mb /data/bucket iceberg: - image: tabulario/iceberg-rest + image: apache/iceberg-rest-fixture depends_on: - create-bucket hostname: iceberg @@ -50,6 +50,7 @@ services: environment: - AWS_REGION=us-east-1 - CATALOG_WAREHOUSE=s3://bucket/warehouse/ + - CATALOG_URI=jdbc:sqlite:file:/tmp/iceberg_rest_mode=memory - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO - CATALOG_S3_ENDPOINT=http://minio:9000 - CATALOG_S3_PATH__STYLE__ACCESS=true diff --git a/site/docs/spark-quickstart.md b/site/docs/spark-quickstart.md index e98bedb49825..7a6e8ccc84ef 100644 --- a/site/docs/spark-quickstart.md +++ b/site/docs/spark-quickstart.md @@ -61,7 +61,7 @@ services: - 10000:10000 - 10001:10001 rest: - image: tabulario/iceberg-rest + image: apache/iceberg-rest-fixture container_name: iceberg-rest networks: iceberg_net: From 5c170ae91b35e01f82d963a0459ec1f0f77643aa Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 17 Dec 2024 22:50:58 +0800 Subject: [PATCH 264/313] docs: Default value of table level distribution-mode should be not set (#11663) --- docs/docs/configuration.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index 100c4d35771e..c784566ef5f5 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -67,10 +67,10 @@ Iceberg tables support table properties to configure table behavior, like the de | write.metadata.metrics.column.col1 | (not set) | Metrics mode for column 'col1' to allow per-column tuning; none, counts, truncate(length), or full | | write.target-file-size-bytes | 536870912 (512 MB) | Controls the size of files generated to target about this many bytes | | write.delete.target-file-size-bytes | 67108864 (64 MB) | Controls the size of delete files generated to target about this many bytes | -| write.distribution-mode | none, see engines for specific defaults, for example [Spark Writes](spark-writes.md#writing-distribution-modes) | Defines distribution of write data: __none__: don't shuffle rows; __hash__: hash distribute by partition key ; __range__: range distribute by partition key or sort key if table has an SortOrder | -| write.delete.distribution-mode | hash | Defines distribution of write delete data | -| write.update.distribution-mode | hash | Defines distribution of write update data | -| write.merge.distribution-mode | none | Defines distribution of write merge data | +| write.distribution-mode | not set, see engines for specific defaults, for example [Spark Writes](spark-writes.md#writing-distribution-modes) | Defines distribution of write data: __none__: don't shuffle rows; __hash__: hash distribute by partition key ; __range__: range distribute by partition key or sort key if table has an SortOrder | +| write.delete.distribution-mode | (not set) | Defines distribution of write delete data | +| write.update.distribution-mode | (not set) | Defines distribution of write update data | +| write.merge.distribution-mode | (not set) | Defines distribution of write merge data | | write.wap.enabled | false | Enables write-audit-publish writes | | write.summary.partition-limit | 0 | Includes partition-level summary stats in snapshot summaries if the changed partition count is less than this limit | | write.metadata.delete-after-commit.enabled | false | Controls whether to delete the oldest **tracked** version metadata files after commit | From 3adcd89061ae0f4db7d53ad5bc548464a27f2b27 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 17 Dec 2024 23:09:30 +0800 Subject: [PATCH 265/313] Docs: Fix Spark catalog `table-override` description (#11684) --- docs/docs/spark-configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 8342d1d57727..5a44c84a1606 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -76,7 +76,7 @@ Both catalogs are configured using properties nested under the catalog name. Com | spark.sql.catalog._catalog-name_.cache-enabled | `true` or `false` | Whether to enable catalog cache, default value is `true` | | spark.sql.catalog._catalog-name_.cache.expiration-interval-ms | `30000` (30 seconds) | Duration after which cached catalog entries are expired; Only effective if `cache-enabled` is `true`. `-1` disables cache expiration and `0` disables caching entirely, irrespective of `cache-enabled`. Default is `30000` (30 seconds) | | spark.sql.catalog._catalog-name_.table-default._propertyKey_ | | Default Iceberg table property value for property key _propertyKey_, which will be set on tables created by this catalog if not overridden | -| spark.sql.catalog._catalog-name_.table-override._propertyKey_ | | Enforced Iceberg table property value for property key _propertyKey_, which cannot be overridden by user | +| spark.sql.catalog._catalog-name_.table-override._propertyKey_ | | Enforced Iceberg table property value for property key _propertyKey_, which cannot be overridden on table creation by user | | spark.sql.catalog._catalog-name_.use-nullable-query-schema | `true` or `false` | Whether to preserve fields' nullability when creating the table using CTAS and RTAS. If set to `true`, all fields will be marked as nullable. If set to `false`, fields' nullability will be preserved. The default value is `true`. Available in Spark 3.5 and above. | Additional properties can be found in common [catalog configuration](configuration.md#catalog-properties). From ce7a4b42f466ffc272f552f1a63b755f20a8dcf7 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 17 Dec 2024 18:16:55 +0100 Subject: [PATCH 266/313] API: Add missing deprecations (#11734) --- .../org/apache/iceberg/transforms/Bucket.java | 14 ++++++++++++++ .../org/apache/iceberg/transforms/Dates.java | 8 ++++++++ .../apache/iceberg/transforms/Identity.java | 8 ++++++++ .../apache/iceberg/transforms/Timestamps.java | 8 ++++++++ .../iceberg/transforms/UnknownTransform.java | 9 +++++++++ .../iceberg/transforms/VoidTransform.java | 18 ++++++++++++++++++ 6 files changed, 65 insertions(+) diff --git a/api/src/main/java/org/apache/iceberg/transforms/Bucket.java b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java index 0e4e782cc110..2b2439e3ed0a 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Bucket.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java @@ -43,6 +43,12 @@ static Bucket get(int numBuckets) { return new Bucket<>(numBuckets); } + /** + * Instantiates a new Bucket Transform + * + * @deprecated will be removed in 2.0.0; use {@link #get(int)} instead + */ + @Deprecated @SuppressWarnings("unchecked") static & SerializableFunction> B get( Type type, int numBuckets) { @@ -94,6 +100,14 @@ protected int hash(T value) { "hash(value) is not supported on the base Bucket class"); } + /** + * Transforms a value to its corresponding partition value. + * + * @param value a source value + * @return a transformed partition value + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public Integer apply(T value) { if (value == null) { diff --git a/api/src/main/java/org/apache/iceberg/transforms/Dates.java b/api/src/main/java/org/apache/iceberg/transforms/Dates.java index 88db16797867..841e6dfa3a51 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Dates.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Dates.java @@ -73,6 +73,14 @@ public Integer apply(Integer days) { this.apply = new Apply(granularity); } + /** + * Transforms a value to its corresponding partition value. + * + * @param days a source value + * @return a transformed partition value + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public Integer apply(Integer days) { return apply.apply(days); diff --git a/api/src/main/java/org/apache/iceberg/transforms/Identity.java b/api/src/main/java/org/apache/iceberg/transforms/Identity.java index 04f0c25e9222..099a99cc3cf4 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Identity.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Identity.java @@ -71,6 +71,14 @@ private Identity(Type type) { this.type = type; } + /** + * Transforms a value to its corresponding partition value. + * + * @param value a source value + * @return a transformed partition value + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public T apply(T value) { return value; diff --git a/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java index 8b8c2ca0a96b..f2c705506305 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java @@ -52,6 +52,14 @@ enum Timestamps implements Transform { this.apply = apply; } + /** + * Transforms a value to its corresponding partition value. + * + * @param timestamp a source value + * @return a transformed partition value + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public Integer apply(Long timestamp) { return apply.apply(timestamp); diff --git a/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java b/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java index c176fd766a35..aebd3445e36e 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/UnknownTransform.java @@ -33,6 +33,15 @@ public class UnknownTransform implements Transform { this.transform = transform; } + /** + * Transforms a value to its corresponding partition value. + * + * @param value a source value + * @return ∅ + * @throws UnsupportedOperationException Implementation is unknown + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public T apply(S value) { throw new UnsupportedOperationException( diff --git a/api/src/main/java/org/apache/iceberg/transforms/VoidTransform.java b/api/src/main/java/org/apache/iceberg/transforms/VoidTransform.java index 5e8e7494c4b5..b46780244faf 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/VoidTransform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/VoidTransform.java @@ -49,6 +49,14 @@ public Void apply(S t) { private VoidTransform() {} + /** + * Transforms a value to its corresponding partition value. + * + * @param value a source value + * @return null + * @deprecated will be removed in 2.0.0; use {@link #bind(Type)} instead + */ + @Deprecated @Override public Void apply(Object value) { return null; @@ -84,6 +92,16 @@ public boolean isVoid() { return true; } + /** + * Returns a human-readable String representation of a transformed value. + * + *

    null values will return "null" + * + * @param value a transformed value + * @return a human-readable String representation of null + * @deprecated will be removed in 2.0.0; use {@link #toHumanString(Type, Object)} instead + */ + @Deprecated @Override public String toHumanString(Void value) { return "null"; From ed06c9cad8ecfd6cc1c0b9e11e11d509428ba0db Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Wed, 18 Dec 2024 04:06:54 +0800 Subject: [PATCH 267/313] Core, Spark 3.5: Fix test failures due to timeout (#11654) --- .../apache/iceberg/hadoop/TestHadoopCommits.java | 14 +++++++++++++- .../iceberg/spark/extensions/TestDelete.java | 12 ++++++++++-- .../apache/iceberg/spark/extensions/TestMerge.java | 12 ++++++++++-- 3 files changed, 33 insertions(+), 5 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java index a8139180ca7d..87ae72431726 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.hadoop; +import static org.apache.iceberg.CatalogProperties.LOCK_ACQUIRE_TIMEOUT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; @@ -421,7 +424,16 @@ public void testConcurrentFastAppends(@TempDir File dir) throws Exception { TABLES.create( SCHEMA, SPEC, - ImmutableMap.of(COMMIT_NUM_RETRIES, String.valueOf(threadsCount)), + ImmutableMap.of( + COMMIT_NUM_RETRIES, + String.valueOf(threadsCount), + COMMIT_MIN_RETRY_WAIT_MS, + "10", + COMMIT_MAX_RETRY_WAIT_MS, + "1000", + // Disable extra retry on lock acquire failure since commit will fail anyway. + LOCK_ACQUIRE_TIMEOUT_MS, + "0"), dir.toURI().toString()); String fileName = UUID.randomUUID().toString(); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index c39ef335ea39..5e42c2dabb41 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -23,6 +23,8 @@ import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; import static org.apache.iceberg.SnapshotSummary.ADDED_DVS_PROP; import static org.apache.iceberg.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.DELETE_DISTRIBUTION_MODE; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; import static org.apache.iceberg.TableProperties.DELETE_MODE; @@ -1148,8 +1150,14 @@ public synchronized void testDeleteWithSnapshotIsolation() createOrReplaceView("deleted_id", Collections.singletonList(1), Encoders.INT()); sql( - "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", - tableName, DELETE_ISOLATION_LEVEL, "snapshot"); + "ALTER TABLE %s SET TBLPROPERTIES('%s'='%s', '%s'='%s', '%s'='%s')", + tableName, + DELETE_ISOLATION_LEVEL, + "snapshot", + COMMIT_MIN_RETRY_WAIT_MS, + "10", + COMMIT_MAX_RETRY_WAIT_MS, + "1000"); sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); createBranchIfNeeded(); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java index 96966a2bea6f..5304e6f752df 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.MERGE_DISTRIBUTION_MODE; import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; import static org.apache.iceberg.TableProperties.MERGE_MODE; @@ -1610,8 +1612,14 @@ public synchronized void testMergeWithSnapshotIsolation() createOrReplaceView("source", Collections.singletonList(1), Encoders.INT()); sql( - "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", - tableName, MERGE_ISOLATION_LEVEL, "snapshot"); + "ALTER TABLE %s SET TBLPROPERTIES('%s'='%s', '%s'='%s', '%s'='%s')", + tableName, + MERGE_ISOLATION_LEVEL, + "snapshot", + COMMIT_MIN_RETRY_WAIT_MS, + "10", + COMMIT_MAX_RETRY_WAIT_MS, + "1000"); sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); createBranchIfNeeded(); From a6cfc12ab080e3dafcec05e10db665c6a843fe4c Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Wed, 18 Dec 2024 00:11:34 +0100 Subject: [PATCH 268/313] Auth Manager API part 1: HTTPRequest, HTTPHeader (#11769) * Auth Manager API part 1: HTTPRequest, HTTPHeader * review * remove static methods * verify error messages * checkstyle * review * review * review --- .../org/apache/iceberg/rest/HTTPHeaders.java | 110 +++++++++++ .../org/apache/iceberg/rest/HTTPRequest.java | 126 ++++++++++++ .../apache/iceberg/rest/TestHTTPHeaders.java | 137 +++++++++++++ .../apache/iceberg/rest/TestHTTPRequest.java | 187 ++++++++++++++++++ 4 files changed, 560 insertions(+) create mode 100644 core/src/main/java/org/apache/iceberg/rest/HTTPHeaders.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/HTTPRequest.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/TestHTTPHeaders.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/TestHTTPRequest.java diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPHeaders.java b/core/src/main/java/org/apache/iceberg/rest/HTTPHeaders.java new file mode 100644 index 000000000000..35710bd9a9b7 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPHeaders.java @@ -0,0 +1,110 @@ +/* + * 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.rest; + +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.immutables.value.Value; + +/** + * Represents a group of HTTP headers. + * + *

    Header name comparison in this class is always case-insensitive, in accordance with RFC 2616. + * + *

    This class exposes methods to convert to and from different representations such as maps and + * multimap, for easier access and manipulation – especially when dealing with multiple headers with + * the same name. + */ +@Value.Style(depluralize = true) +@Value.Immutable +@SuppressWarnings({"ImmutablesStyle", "SafeLoggingPropagation"}) +public interface HTTPHeaders { + + HTTPHeaders EMPTY = of(); + + /** Returns all the header entries in this group. */ + Set entries(); + + /** Returns all the entries in this group for the given name (case-insensitive). */ + default Set entries(String name) { + return entries().stream() + .filter(header -> header.name().equalsIgnoreCase(name)) + .collect(Collectors.toSet()); + } + + /** Returns whether this group contains an entry with the given name (case-insensitive). */ + default boolean contains(String name) { + return entries().stream().anyMatch(header -> header.name().equalsIgnoreCase(name)); + } + + /** + * Adds the given header to the current group if no entry with the same name is already present. + * Returns a new instance with the added header, or the current instance if the header is already + * present. + */ + default HTTPHeaders putIfAbsent(HTTPHeader header) { + Preconditions.checkNotNull(header, "header"); + return contains(header.name()) + ? this + : ImmutableHTTPHeaders.builder().from(this).addEntry(header).build(); + } + + /** + * Adds the given headers to the current group if no entries with same names are already present. + * Returns a new instance with the added headers, or the current instance if all headers are + * already present. + */ + default HTTPHeaders putIfAbsent(HTTPHeaders headers) { + Preconditions.checkNotNull(headers, "headers"); + List newHeaders = + headers.entries().stream().filter(e -> !contains(e.name())).collect(Collectors.toList()); + return newHeaders.isEmpty() + ? this + : ImmutableHTTPHeaders.builder().from(this).addAllEntries(newHeaders).build(); + } + + static HTTPHeaders of(HTTPHeader... headers) { + return ImmutableHTTPHeaders.builder().addEntries(headers).build(); + } + + /** Represents an HTTP header as a name-value pair. */ + @Value.Style(redactedMask = "****", depluralize = true) + @Value.Immutable + @SuppressWarnings({"ImmutablesStyle", "SafeLoggingPropagation"}) + interface HTTPHeader { + + String name(); + + @Value.Redacted + String value(); + + @Value.Check + default void check() { + if (name().isEmpty()) { + throw new IllegalArgumentException("Header name cannot be empty"); + } + } + + static HTTPHeader of(String name, String value) { + return ImmutableHTTPHeader.builder().name(name).value(value).build(); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPRequest.java b/core/src/main/java/org/apache/iceberg/rest/HTTPRequest.java new file mode 100644 index 000000000000..41921d946ca8 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPRequest.java @@ -0,0 +1,126 @@ +/* + * 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.rest; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.hc.core5.net.URIBuilder; +import org.apache.iceberg.exceptions.RESTException; +import org.immutables.value.Value; + +/** Represents an HTTP request. */ +@Value.Style(redactedMask = "****", depluralize = true) +@Value.Immutable +@SuppressWarnings({"ImmutablesStyle", "SafeLoggingPropagation"}) +public interface HTTPRequest { + + enum HTTPMethod { + GET, + HEAD, + POST, + DELETE + } + + /** + * Returns the base URI configured at the REST client level. The base URI is used to construct the + * full {@link #requestUri()}. + */ + URI baseUri(); + + /** + * Returns the full URI of this request. The URI is constructed from the base URI, path, and query + * parameters. It cannot be modified directly. + */ + @Value.Lazy + default URI requestUri() { + // if full path is provided, use the input path as path + String fullPath = + (path().startsWith("https://") || path().startsWith("http://")) + ? path() + : String.format("%s/%s", baseUri(), path()); + try { + URIBuilder builder = new URIBuilder(RESTUtil.stripTrailingSlash(fullPath)); + queryParameters().forEach(builder::addParameter); + return builder.build(); + } catch (URISyntaxException e) { + throw new RESTException( + "Failed to create request URI from base %s, params %s", fullPath, queryParameters()); + } + } + + /** Returns the HTTP method of this request. */ + HTTPMethod method(); + + /** Returns the path of this request. */ + String path(); + + /** Returns the query parameters of this request. */ + Map queryParameters(); + + /** Returns the headers of this request. */ + @Value.Default + default HTTPHeaders headers() { + return HTTPHeaders.EMPTY; + } + + /** Returns the raw, unencoded request body. */ + @Nullable + @Value.Redacted + Object body(); + + /** Returns the encoded request body as a string. */ + @Value.Lazy + @Nullable + @Value.Redacted + default String encodedBody() { + Object body = body(); + if (body instanceof Map) { + return RESTUtil.encodeFormData((Map) body); + } else if (body != null) { + try { + return mapper().writeValueAsString(body); + } catch (JsonProcessingException e) { + throw new RESTException(e, "Failed to encode request body: %s", body); + } + } + return null; + } + + /** + * Returns the {@link ObjectMapper} to use for encoding the request body. The default is {@link + * RESTObjectMapper#mapper()}. + */ + @Value.Default + default ObjectMapper mapper() { + return RESTObjectMapper.mapper(); + } + + @Value.Check + default void check() { + if (path().startsWith("/")) { + throw new RESTException( + "Received a malformed path for a REST request: %s. Paths should not start with /", + path()); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/TestHTTPHeaders.java b/core/src/test/java/org/apache/iceberg/rest/TestHTTPHeaders.java new file mode 100644 index 000000000000..9380073f7643 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/TestHTTPHeaders.java @@ -0,0 +1,137 @@ +/* + * 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.rest; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.rest.HTTPHeaders.HTTPHeader; +import org.junit.jupiter.api.Test; + +class TestHTTPHeaders { + + private final HTTPHeaders headers = + HTTPHeaders.of( + HTTPHeader.of("header1", "value1a"), + HTTPHeader.of("HEADER1", "value1b"), + HTTPHeader.of("header2", "value2")); + + @Test + void entries() { + assertThat(headers.entries()) + .containsExactlyInAnyOrder( + HTTPHeader.of("header1", "value1a"), + HTTPHeader.of("HEADER1", "value1b"), + HTTPHeader.of("header2", "value2")); + + // duplicated entries + assertThat( + HTTPHeaders.of(HTTPHeader.of("header1", "value1"), HTTPHeader.of("header1", "value1")) + .entries()) + .containsExactly(HTTPHeader.of("header1", "value1")); + } + + @Test + void entriesByName() { + assertThat(headers.entries("header1")) + .containsExactlyInAnyOrder( + HTTPHeader.of("header1", "value1a"), HTTPHeader.of("HEADER1", "value1b")); + assertThat(headers.entries("HEADER1")) + .containsExactlyInAnyOrder( + HTTPHeader.of("header1", "value1a"), HTTPHeader.of("HEADER1", "value1b")); + assertThat(headers.entries("header2")) + .containsExactlyInAnyOrder(HTTPHeader.of("header2", "value2")); + assertThat(headers.entries("HEADER2")) + .containsExactlyInAnyOrder(HTTPHeader.of("header2", "value2")); + assertThat(headers.entries("header3")).isEmpty(); + assertThat(headers.entries("HEADER3")).isEmpty(); + assertThat(headers.entries(null)).isEmpty(); + } + + @Test + void contains() { + assertThat(headers.contains("header1")).isTrue(); + assertThat(headers.contains("HEADER1")).isTrue(); + assertThat(headers.contains("header2")).isTrue(); + assertThat(headers.contains("HEADER2")).isTrue(); + assertThat(headers.contains("header3")).isFalse(); + assertThat(headers.contains("HEADER3")).isFalse(); + assertThat(headers.contains(null)).isFalse(); + } + + @Test + void putIfAbsentHTTPHeader() { + HTTPHeaders actual = headers.putIfAbsent(HTTPHeader.of("Header1", "value1c")); + assertThat(actual).isSameAs(headers); + + actual = headers.putIfAbsent(HTTPHeader.of("header3", "value3")); + assertThat(actual.entries()) + .containsExactly( + HTTPHeader.of("header1", "value1a"), + HTTPHeader.of("HEADER1", "value1b"), + HTTPHeader.of("header2", "value2"), + HTTPHeader.of("header3", "value3")); + + assertThatThrownBy(() -> headers.putIfAbsent((HTTPHeader) null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("header"); + } + + @Test + void putIfAbsentHTTPHeaders() { + HTTPHeaders actual = headers.putIfAbsent(HTTPHeaders.of(HTTPHeader.of("Header1", "value1c"))); + assertThat(actual).isSameAs(headers); + + actual = + headers.putIfAbsent( + ImmutableHTTPHeaders.builder() + .addEntry(HTTPHeader.of("Header1", "value1c")) + .addEntry(HTTPHeader.of("header3", "value3")) + .build()); + assertThat(actual) + .isEqualTo( + ImmutableHTTPHeaders.builder() + .addEntries( + HTTPHeader.of("header1", "value1a"), + HTTPHeader.of("HEADER1", "value1b"), + HTTPHeader.of("header2", "value2"), + HTTPHeader.of("header3", "value3")) + .build()); + + assertThatThrownBy(() -> headers.putIfAbsent((HTTPHeaders) null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("headers"); + } + + @Test + void invalidHeader() { + // invalid input (null name or value) + assertThatThrownBy(() -> HTTPHeader.of(null, "value1")) + .isInstanceOf(NullPointerException.class) + .hasMessage("name"); + assertThatThrownBy(() -> HTTPHeader.of("header1", null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("value"); + + // invalid input (empty name) + assertThatThrownBy(() -> HTTPHeader.of("", "value1")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Header name cannot be empty"); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/TestHTTPRequest.java b/core/src/test/java/org/apache/iceberg/rest/TestHTTPRequest.java new file mode 100644 index 000000000000..84e1b0830c9b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/TestHTTPRequest.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.net.URI; +import java.util.Map; +import java.util.stream.Stream; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.exceptions.RESTException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.rest.requests.CreateNamespaceRequest; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; + +class TestHTTPRequest { + + @ParameterizedTest + @MethodSource("validRequestUris") + public void requestUriSuccess(HTTPRequest request, URI expected) { + assertThat(request.requestUri()).isEqualTo(expected); + } + + public static Stream validRequestUris() { + return Stream.of( + Arguments.of( + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost:8080/foo")) + .method(HTTPRequest.HTTPMethod.GET) + .path("v1/namespaces/ns/tables/") // trailing slash should be removed + .putQueryParameter("pageToken", "1234") + .putQueryParameter("pageSize", "10") + .build(), + URI.create( + "http://localhost:8080/foo/v1/namespaces/ns/tables?pageToken=1234&pageSize=10")), + Arguments.of( + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost:8080/foo")) + .method(HTTPRequest.HTTPMethod.GET) + .path("https://authserver.com/token") // absolute path HTTPS + .build(), + URI.create("https://authserver.com/token")), + Arguments.of( + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost:8080/foo")) + .method(HTTPRequest.HTTPMethod.GET) + .path("http://authserver.com/token") // absolute path HTTP + .build(), + URI.create("http://authserver.com/token"))); + } + + @Test + public void malformedPath() { + assertThatThrownBy( + () -> + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost")) + .method(HTTPRequest.HTTPMethod.GET) + .path("/v1/namespaces") // wrong leading slash + .build()) + .isInstanceOf(RESTException.class) + .hasMessage( + "Received a malformed path for a REST request: /v1/namespaces. Paths should not start with /"); + } + + @Test + public void invalidPath() { + HTTPRequest request = + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost")) + .method(HTTPRequest.HTTPMethod.GET) + .path(" not a valid path") // wrong path + .build(); + assertThatThrownBy(request::requestUri) + .isInstanceOf(RESTException.class) + .hasMessage( + "Failed to create request URI from base http://localhost/ not a valid path, params {}"); + } + + @Test + public void encodedBodyJSON() { + HTTPRequest request = + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost")) + .method(HTTPRequest.HTTPMethod.POST) + .path("v1/namespaces/ns") + .body( + CreateNamespaceRequest.builder() + .withNamespace(Namespace.of("ns")) + .setProperties(ImmutableMap.of("prop1", "value1")) + .build()) + .build(); + assertThat(request.encodedBody()) + .isEqualTo("{\"namespace\":[\"ns\"],\"properties\":{\"prop1\":\"value1\"}}"); + } + + @Test + public void encodedBodyJSONInvalid() throws JsonProcessingException { + ObjectMapper mapper = Mockito.mock(ObjectMapper.class); + Mockito.when(mapper.writeValueAsString(Mockito.any())) + .thenThrow(new JsonMappingException(null, "invalid")); + HTTPRequest request = + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost")) + .method(HTTPRequest.HTTPMethod.POST) + .path("token") + .body("invalid") + .mapper(mapper) + .build(); + assertThatThrownBy(request::encodedBody) + .isInstanceOf(RESTException.class) + .hasMessage("Failed to encode request body: invalid"); + } + + @Test + public void encodedBodyFormData() { + HTTPRequest request = + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost")) + .method(HTTPRequest.HTTPMethod.POST) + .path("token") + .body( + ImmutableMap.of( + "grant_type", "urn:ietf:params:oauth:grant-type:token-exchange", + "subject_token", "token", + "subject_token_type", "urn:ietf:params:oauth:token-type:access_token", + "scope", "catalog")) + .build(); + assertThat(request.encodedBody()) + .isEqualTo( + "grant_type=urn%3Aietf%3Aparams%3Aoauth%3Agrant-type%3Atoken-exchange&" + + "subject_token=token&" + + "subject_token_type=urn%3Aietf%3Aparams%3Aoauth%3Atoken-type%3Aaccess_token&" + + "scope=catalog"); + } + + @Test + public void encodedBodyFormDataNullKeysAndValues() { + Map body = Maps.newHashMap(); + body.put(null, "token"); + body.put("scope", null); + HTTPRequest request = + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost")) + .method(HTTPRequest.HTTPMethod.POST) + .path("token") + .body(body) + .build(); + assertThat(request.encodedBody()).isEqualTo("null=token&scope=null"); + } + + @Test + public void encodedBodyNull() { + HTTPRequest request = + ImmutableHTTPRequest.builder() + .baseUri(URI.create("http://localhost")) + .method(HTTPRequest.HTTPMethod.POST) + .path("token") + .build(); + assertThat(request.encodedBody()).isNull(); + } +} From e3628c18c2009e796ee404ac31c994e3a90b268b Mon Sep 17 00:00:00 2001 From: big face cat <731030576@qq.com> Date: Wed, 18 Dec 2024 14:08:15 +0800 Subject: [PATCH 269/313] =?UTF-8?q?Flink:=20make=20`StatisticsOrRecord`=20?= =?UTF-8?q?to=20be=20correctly=20serialized=20and=20deser=E2=80=A6=20(#115?= =?UTF-8?q?57)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: huyuanfeng --- .../apache/iceberg/flink/sink/FlinkSink.java | 5 +- .../StatisticsOrRecordTypeInformation.java | 115 ++++++++++++++++++ ...TestStatisticsOrRecordTypeInformation.java | 46 +++++++ 3 files changed, 165 insertions(+), 1 deletion(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestStatisticsOrRecordTypeInformation.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 2e586b960c22..18f3557beeff 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -65,6 +65,7 @@ import org.apache.iceberg.flink.sink.shuffle.DataStatisticsOperatorFactory; import org.apache.iceberg.flink.sink.shuffle.RangePartitioner; import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; +import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecordTypeInformation; import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; @@ -644,12 +645,14 @@ private DataStream distributeDataStream( } LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsOrRecordTypeInformation statisticsOrRecordTypeInformation = + new StatisticsOrRecordTypeInformation(flinkRowType, iSchema, sortOrder); StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); SingleOutputStreamOperator shuffleStream = input .transform( operatorName("range-shuffle"), - TypeInformation.of(StatisticsOrRecord.class), + statisticsOrRecordTypeInformation, new DataStatisticsOperatorFactory( iSchema, sortOrder, diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java new file mode 100644 index 000000000000..921ede9466e0 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordTypeInformation.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Objects; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; + +public class StatisticsOrRecordTypeInformation extends TypeInformation { + + private final TypeInformation rowTypeInformation; + private final SortOrder sortOrder; + private final GlobalStatisticsSerializer globalStatisticsSerializer; + + public StatisticsOrRecordTypeInformation( + RowType flinkRowType, Schema schema, SortOrder sortOrder) { + this.sortOrder = sortOrder; + this.rowTypeInformation = FlinkCompatibilityUtil.toTypeInfo(flinkRowType); + this.globalStatisticsSerializer = + new GlobalStatisticsSerializer(new SortKeySerializer(schema, sortOrder)); + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 1; + } + + @Override + public int getTotalFields() { + return 1; + } + + @Override + public Class getTypeClass() { + return StatisticsOrRecord.class; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(SerializerConfig config) { + TypeSerializer recordSerializer = rowTypeInformation.createSerializer(config); + return new StatisticsOrRecordSerializer(globalStatisticsSerializer, recordSerializer); + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig config) { + return createSerializer(config.getSerializerConfig()); + } + + @Override + public String toString() { + return "StatisticsOrRecord"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (o != null && this.getClass() == o.getClass()) { + StatisticsOrRecordTypeInformation that = (StatisticsOrRecordTypeInformation) o; + return that.sortOrder.equals(sortOrder) + && that.rowTypeInformation.equals(rowTypeInformation) + && that.globalStatisticsSerializer.equals(globalStatisticsSerializer); + } else { + return false; + } + } + + @Override + public int hashCode() { + return Objects.hash(rowTypeInformation, sortOrder, globalStatisticsSerializer); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof StatisticsOrRecordTypeInformation; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestStatisticsOrRecordTypeInformation.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestStatisticsOrRecordTypeInformation.java new file mode 100644 index 000000000000..f54198522e99 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestStatisticsOrRecordTypeInformation.java @@ -0,0 +1,46 @@ +/* + * 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.flink.sink.shuffle; + +import org.apache.flink.api.common.typeutils.TypeInformationTestBase; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.types.Types; + +public class TestStatisticsOrRecordTypeInformation + extends TypeInformationTestBase { + private static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "uuid", Types.UUIDType.get()), + Types.NestedField.optional(3, "data", Types.StringType.get())); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + private static final SortOrder SORT_ORDER1 = SortOrder.builderFor(SCHEMA).asc("ts").build(); + private static final SortOrder SORT_ORDER2 = SortOrder.builderFor(SCHEMA).asc("data").build(); + + @Override + protected StatisticsOrRecordTypeInformation[] getTestData() { + return new StatisticsOrRecordTypeInformation[] { + new StatisticsOrRecordTypeInformation(ROW_TYPE, SCHEMA, SORT_ORDER1), + new StatisticsOrRecordTypeInformation(ROW_TYPE, SCHEMA, SORT_ORDER2), + }; + } +} From b428fbc59bd1579f4dc918a5cd48fce667d81ce1 Mon Sep 17 00:00:00 2001 From: Ppei-Wang Date: Wed, 18 Dec 2024 14:40:48 +0800 Subject: [PATCH 270/313] Spark 3.4,3.5: Use correct identifier in view DESCRIBE cmd (#11751) --- .../datasources/v2/DescribeV2ViewExec.scala | 4 +- .../iceberg/spark/extensions/TestViews.java | 63 +++++++++++++++++++ .../datasources/v2/DescribeV2ViewExec.scala | 4 +- .../iceberg/spark/extensions/TestViews.java | 63 +++++++++++++++++++ 4 files changed, 132 insertions(+), 2 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala index bb08fb18b2bd..2a5566a15f21 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala @@ -55,13 +55,15 @@ case class DescribeV2ViewExec( private def describeExtended: Seq[InternalRow] = { val outputColumns = view.queryColumnNames.mkString("[", ", ", "]") val properties: Map[String, String] = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala - val viewCatalogAndNamespace: Seq[String] = view.currentCatalog +: view.currentNamespace.toSeq + val viewCatalogAndNamespace: Seq[String] = view.name.split("\\.").take(2) val viewProperties = properties.toSeq.sortBy(_._1).map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" }.mkString("[", ", ", "]") + // omitting view text here because it is shown as + // part of SHOW CREATE TABLE and can result in weird formatting in the DESCRIBE output toCatalystRow("# Detailed View Information", "", "") :: toCatalystRow("Comment", view.properties.getOrDefault(ViewCatalog.PROP_COMMENT, ""), "") :: toCatalystRow("View Catalog and Namespace", viewCatalogAndNamespace.quoted, "") :: diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index 51d6064d9a14..a9160832aa39 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -45,6 +45,7 @@ import org.apache.iceberg.view.View; import org.apache.iceberg.view.ViewHistoryEntry; import org.apache.iceberg.view.ViewProperties; +import org.apache.iceberg.view.ViewUtil; import org.apache.iceberg.view.ViewVersion; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; @@ -1422,6 +1423,68 @@ public void describeExtendedView() { "")); } + @Test + public void createAndDescribeViewInDefaultNamespace() { + String viewName = viewName("createViewInDefaultNamespace"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + + sql("CREATE VIEW %s (id, data) AS %s", viewName, sql); + TableIdentifier identifier = TableIdentifier.of(NAMESPACE, viewName); + View view = viewCatalog().loadView(identifier); + assertThat(view.currentVersion().defaultCatalog()).isNull(); + assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalogName, identifier)); + assertThat(view.currentVersion().defaultNamespace()).isEqualTo(NAMESPACE); + + String location = viewCatalog().loadView(identifier).location(); + assertThat(sql("DESCRIBE EXTENDED %s.%s", NAMESPACE, viewName)) + .contains( + row("id", "int", ""), + row("data", "string", ""), + row("", "", ""), + row("# Detailed View Information", "", ""), + row("Comment", "", ""), + row("View Catalog and Namespace", String.format("%s.%s", catalogName, NAMESPACE), ""), + row("View Query Output Columns", "[id, data]", ""), + row( + "View Properties", + String.format( + "['format-version' = '1', 'location' = '%s', 'provider' = 'iceberg']", + location), + "")); + } + + @Test + public void createAndDescribeViewWithoutCurrentNamespace() { + String viewName = viewName("createViewWithoutCurrentNamespace"); + Namespace namespace = Namespace.of("test_namespace"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + + sql("CREATE NAMESPACE IF NOT EXISTS %s", namespace); + sql("CREATE VIEW %s.%s (id, data) AS %s", namespace, viewName, sql); + TableIdentifier identifier = TableIdentifier.of(namespace, viewName); + View view = viewCatalog().loadView(identifier); + assertThat(view.currentVersion().defaultCatalog()).isNull(); + assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalogName, identifier)); + assertThat(view.currentVersion().defaultNamespace()).isEqualTo(NAMESPACE); + + String location = viewCatalog().loadView(identifier).location(); + assertThat(sql("DESCRIBE EXTENDED %s.%s", namespace, viewName)) + .contains( + row("id", "int", ""), + row("data", "string", ""), + row("", "", ""), + row("# Detailed View Information", "", ""), + row("Comment", "", ""), + row("View Catalog and Namespace", String.format("%s.%s", catalogName, namespace), ""), + row("View Query Output Columns", "[id, data]", ""), + row( + "View Properties", + String.format( + "['format-version' = '1', 'location' = '%s', 'provider' = 'iceberg']", + location), + "")); + } + @Test public void showViewProperties() { String viewName = viewName("showViewProps"); diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala index bb08fb18b2bd..2a5566a15f21 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala @@ -55,13 +55,15 @@ case class DescribeV2ViewExec( private def describeExtended: Seq[InternalRow] = { val outputColumns = view.queryColumnNames.mkString("[", ", ", "]") val properties: Map[String, String] = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala - val viewCatalogAndNamespace: Seq[String] = view.currentCatalog +: view.currentNamespace.toSeq + val viewCatalogAndNamespace: Seq[String] = view.name.split("\\.").take(2) val viewProperties = properties.toSeq.sortBy(_._1).map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" }.mkString("[", ", ", "]") + // omitting view text here because it is shown as + // part of SHOW CREATE TABLE and can result in weird formatting in the DESCRIBE output toCatalystRow("# Detailed View Information", "", "") :: toCatalystRow("Comment", view.properties.getOrDefault(ViewCatalog.PROP_COMMENT, ""), "") :: toCatalystRow("View Catalog and Namespace", viewCatalogAndNamespace.quoted, "") :: diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index 03a31d44d69d..f22fbf936908 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -50,6 +50,7 @@ import org.apache.iceberg.view.View; import org.apache.iceberg.view.ViewHistoryEntry; import org.apache.iceberg.view.ViewProperties; +import org.apache.iceberg.view.ViewUtil; import org.apache.iceberg.view.ViewVersion; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; @@ -1451,6 +1452,68 @@ public void describeExtendedView() { "")); } + @TestTemplate + public void createAndDescribeViewInDefaultNamespace() { + String viewName = viewName("createViewInDefaultNamespace"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + + sql("CREATE VIEW %s (id, data) AS %s", viewName, sql); + TableIdentifier identifier = TableIdentifier.of(NAMESPACE, viewName); + View view = viewCatalog().loadView(identifier); + assertThat(view.currentVersion().defaultCatalog()).isNull(); + assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalogName, identifier)); + assertThat(view.currentVersion().defaultNamespace()).isEqualTo(NAMESPACE); + + String location = viewCatalog().loadView(identifier).location(); + assertThat(sql("DESCRIBE EXTENDED %s.%s", NAMESPACE, viewName)) + .contains( + row("id", "int", ""), + row("data", "string", ""), + row("", "", ""), + row("# Detailed View Information", "", ""), + row("Comment", "", ""), + row("View Catalog and Namespace", String.format("%s.%s", catalogName, NAMESPACE), ""), + row("View Query Output Columns", "[id, data]", ""), + row( + "View Properties", + String.format( + "['format-version' = '1', 'location' = '%s', 'provider' = 'iceberg']", + location), + "")); + } + + @TestTemplate + public void createAndDescribeViewWithoutCurrentNamespace() { + String viewName = viewName("createViewWithoutCurrentNamespace"); + Namespace namespace = Namespace.of("test_namespace"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + + sql("CREATE NAMESPACE IF NOT EXISTS %s", namespace); + sql("CREATE VIEW %s.%s (id, data) AS %s", namespace, viewName, sql); + TableIdentifier identifier = TableIdentifier.of(namespace, viewName); + View view = viewCatalog().loadView(identifier); + assertThat(view.currentVersion().defaultCatalog()).isNull(); + assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalogName, identifier)); + assertThat(view.currentVersion().defaultNamespace()).isEqualTo(NAMESPACE); + + String location = viewCatalog().loadView(identifier).location(); + assertThat(sql("DESCRIBE EXTENDED %s.%s", namespace, viewName)) + .contains( + row("id", "int", ""), + row("data", "string", ""), + row("", "", ""), + row("# Detailed View Information", "", ""), + row("Comment", "", ""), + row("View Catalog and Namespace", String.format("%s.%s", catalogName, namespace), ""), + row("View Query Output Columns", "[id, data]", ""), + row( + "View Properties", + String.format( + "['format-version' = '1', 'location' = '%s', 'provider' = 'iceberg']", + location), + "")); + } + @TestTemplate public void showViewProperties() { String viewName = viewName("showViewProps"); From 204a49ca3c9666eb1d0b225ae73cc96ebb9c1c6d Mon Sep 17 00:00:00 2001 From: Karol Sobczak Date: Wed, 18 Dec 2024 16:35:48 +0100 Subject: [PATCH 271/313] Use try-with-resources in TestParallelIterable (#11810) --- .../iceberg/util/TestParallelIterable.java | 292 +++++++++--------- 1 file changed, 153 insertions(+), 139 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java index 5e37e0390db9..410e33058d0c 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -44,165 +44,179 @@ public class TestParallelIterable { @Test public void closeParallelIteratorWithoutCompleteIteration() { ExecutorService executor = Executors.newFixedThreadPool(1); - - Iterable> transform = - Iterables.transform( - Lists.newArrayList(1, 2, 3, 4, 5), - item -> - new CloseableIterable() { - @Override - public void close() {} - - @Override - public CloseableIterator iterator() { - return CloseableIterator.withClose(Collections.singletonList(item).iterator()); - } - }); - - ParallelIterable parallelIterable = new ParallelIterable<>(transform, executor); - ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); - - assertThat(iterator.hasNext()).isTrue(); - assertThat(iterator.next()).isNotNull(); - Awaitility.await("Queue is populated") - .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> queueHasElements(iterator)); - iterator.close(); - Awaitility.await("Queue is cleared") - .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> assertThat(iterator.queueSize()).isEqualTo(0)); + try { + Iterable> transform = + Iterables.transform( + Lists.newArrayList(1, 2, 3, 4, 5), + item -> + new CloseableIterable() { + @Override + public void close() {} + + @Override + public CloseableIterator iterator() { + return CloseableIterator.withClose( + Collections.singletonList(item).iterator()); + } + }); + + ParallelIterable parallelIterable = new ParallelIterable<>(transform, executor); + ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); + + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next()).isNotNull(); + Awaitility.await("Queue is populated") + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> queueHasElements(iterator)); + iterator.close(); + Awaitility.await("Queue is cleared") + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> assertThat(iterator.queueSize()).isEqualTo(0)); + } finally { + executor.shutdown(); + } } @Test public void closeMoreDataParallelIteratorWithoutCompleteIteration() { ExecutorService executor = Executors.newFixedThreadPool(1); - Iterator integerIterator = - new Iterator() { - private int number = 1; - - @Override - public boolean hasNext() { - if (number > 1000) { - return false; + try { + Iterator integerIterator = + new Iterator() { + private int number = 1; + + @Override + public boolean hasNext() { + if (number > 1000) { + return false; + } + + number++; + return true; } - number++; - return true; - } - - @Override - public Integer next() { - try { - // sleep to control number generate rate - Thread.sleep(10); - } catch (InterruptedException e) { - // Sleep interrupted, we ignore it! + @Override + public Integer next() { + try { + // sleep to control number generate rate + Thread.sleep(10); + } catch (InterruptedException e) { + // Sleep interrupted, we ignore it! + } + return number; } - return number; - } - }; - Iterable> transform = - Iterables.transform( - Lists.newArrayList(1), - item -> - new CloseableIterable() { - @Override - public void close() {} - - @Override - public CloseableIterator iterator() { - return CloseableIterator.withClose(integerIterator); - } - }); - - ParallelIterable parallelIterable = new ParallelIterable<>(transform, executor); - ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); - - assertThat(iterator.hasNext()).isTrue(); - assertThat(iterator.next()).isNotNull(); - Awaitility.await("Queue is populated") - .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> queueHasElements(iterator)); - iterator.close(); - Awaitility.await("Queue is cleared") - .atMost(5, TimeUnit.SECONDS) - .untilAsserted( - () -> - assertThat(iterator.queueSize()) - .as("Queue is not empty after cleaning") - .isEqualTo(0)); + }; + Iterable> transform = + Iterables.transform( + Lists.newArrayList(1), + item -> + new CloseableIterable() { + @Override + public void close() {} + + @Override + public CloseableIterator iterator() { + return CloseableIterator.withClose(integerIterator); + } + }); + + ParallelIterable parallelIterable = new ParallelIterable<>(transform, executor); + ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); + + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next()).isNotNull(); + Awaitility.await("Queue is populated") + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> queueHasElements(iterator)); + iterator.close(); + Awaitility.await("Queue is cleared") + .atMost(5, TimeUnit.SECONDS) + .untilAsserted( + () -> + assertThat(iterator.queueSize()) + .as("Queue is not empty after cleaning") + .isEqualTo(0)); + } finally { + executor.shutdown(); + } } @Test public void limitQueueSize() { - List> iterables = - ImmutableList.of( - () -> IntStream.range(0, 100).iterator(), - () -> IntStream.range(0, 100).iterator(), - () -> IntStream.range(0, 100).iterator()); - - Multiset expectedValues = - IntStream.range(0, 100) - .boxed() - .flatMap(i -> Stream.of(i, i, i)) - .collect(ImmutableMultiset.toImmutableMultiset()); - - int maxQueueSize = 20; ExecutorService executor = Executors.newCachedThreadPool(); - ParallelIterable parallelIterable = - new ParallelIterable<>(iterables, executor, maxQueueSize); - ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); - - Multiset actualValues = HashMultiset.create(); - - while (iterator.hasNext()) { - assertThat(iterator.queueSize()) - .as("iterator internal queue size") - .isLessThanOrEqualTo(maxQueueSize + iterables.size()); - actualValues.add(iterator.next()); + try { + List> iterables = + ImmutableList.of( + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator()); + + Multiset expectedValues = + IntStream.range(0, 100) + .boxed() + .flatMap(i -> Stream.of(i, i, i)) + .collect(ImmutableMultiset.toImmutableMultiset()); + + int maxQueueSize = 20; + ParallelIterable parallelIterable = + new ParallelIterable<>(iterables, executor, maxQueueSize); + ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); + + Multiset actualValues = HashMultiset.create(); + + while (iterator.hasNext()) { + assertThat(iterator.queueSize()) + .as("iterator internal queue size") + .isLessThanOrEqualTo(maxQueueSize + iterables.size()); + actualValues.add(iterator.next()); + } + + assertThat(actualValues) + .as("multiset of values returned by the iterator") + .isEqualTo(expectedValues); + + iterator.close(); + } finally { + executor.shutdown(); } - - assertThat(actualValues) - .as("multiset of values returned by the iterator") - .isEqualTo(expectedValues); - - iterator.close(); - executor.shutdownNow(); } @Test public void queueSizeOne() { - List> iterables = - ImmutableList.of( - () -> IntStream.range(0, 100).iterator(), - () -> IntStream.range(0, 100).iterator(), - () -> IntStream.range(0, 100).iterator()); - - Multiset expectedValues = - IntStream.range(0, 100) - .boxed() - .flatMap(i -> Stream.of(i, i, i)) - .collect(ImmutableMultiset.toImmutableMultiset()); - ExecutorService executor = Executors.newCachedThreadPool(); - ParallelIterable parallelIterable = new ParallelIterable<>(iterables, executor, 1); - ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); - - Multiset actualValues = HashMultiset.create(); - - while (iterator.hasNext()) { - assertThat(iterator.queueSize()) - .as("iterator internal queue size") - .isLessThanOrEqualTo(1 + iterables.size()); - actualValues.add(iterator.next()); + try { + List> iterables = + ImmutableList.of( + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator()); + + Multiset expectedValues = + IntStream.range(0, 100) + .boxed() + .flatMap(i -> Stream.of(i, i, i)) + .collect(ImmutableMultiset.toImmutableMultiset()); + + ParallelIterable parallelIterable = new ParallelIterable<>(iterables, executor, 1); + ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); + + Multiset actualValues = HashMultiset.create(); + + while (iterator.hasNext()) { + assertThat(iterator.queueSize()) + .as("iterator internal queue size") + .isLessThanOrEqualTo(1 + iterables.size()); + actualValues.add(iterator.next()); + } + + assertThat(actualValues) + .as("multiset of values returned by the iterator") + .isEqualTo(expectedValues); + + iterator.close(); + } finally { + executor.shutdown(); } - - assertThat(actualValues) - .as("multiset of values returned by the iterator") - .isEqualTo(expectedValues); - - iterator.close(); - executor.shutdownNow(); } private void queueHasElements(ParallelIterator iterator) { From 7e1a4c9fedeb679be85a1921ade7995d5ec2cbec Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 18 Dec 2024 08:10:15 -0800 Subject: [PATCH 272/313] Spark 3.5: Support default values in Parquet reader (#11803) --- .../org/apache/iceberg/spark/SparkUtil.java | 66 +++++ .../spark/data/SparkParquetReaders.java | 26 +- .../iceberg/spark/source/BaseReader.java | 62 +---- .../iceberg/spark/data/TestHelpers.java | 22 +- .../spark/data/TestSparkParquetReader.java | 233 +++++++++++++++++- 5 files changed, 329 insertions(+), 80 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java index de06cceb2677..4bd2e9c21551 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark; +import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.sql.Date; import java.sql.Timestamp; import java.util.List; @@ -25,14 +27,20 @@ import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; +import org.apache.avro.generic.GenericData; +import org.apache.avro.util.Utf8; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.transforms.Transform; import org.apache.iceberg.transforms.UnknownTransform; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.Pair; import org.apache.spark.SparkEnv; import org.apache.spark.scheduler.ExecutorCacheTaskLocation; @@ -40,14 +48,17 @@ import org.apache.spark.sql.catalyst.expressions.BoundReference; import org.apache.spark.sql.catalyst.expressions.EqualTo; import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.catalyst.expressions.Literal; import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.StructType; import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.BlockManagerId; import org.apache.spark.storage.BlockManagerMaster; +import org.apache.spark.unsafe.types.UTF8String; import org.joda.time.DateTime; import scala.collection.JavaConverters; import scala.collection.Seq; @@ -268,4 +279,59 @@ private static List toJavaList(Seq seq) { private static String toExecutorLocation(BlockManagerId id) { return ExecutorCacheTaskLocation.apply(id.host(), id.executorId()).toString(); } + + /** + * Converts a value to pass into Spark from Iceberg's internal object model. + * + * @param type an Iceberg type + * @param value a value that is an instance of {@link Type.TypeID#javaClass()} + * @return the value converted for Spark + */ + public static Object convertConstant(Type type, Object value) { + if (value == null) { + return null; + } + + switch (type.typeId()) { + case DECIMAL: + return Decimal.apply((BigDecimal) value); + case STRING: + if (value instanceof Utf8) { + Utf8 utf8 = (Utf8) value; + return UTF8String.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); + } + return UTF8String.fromString(value.toString()); + case FIXED: + if (value instanceof byte[]) { + return value; + } else if (value instanceof GenericData.Fixed) { + return ((GenericData.Fixed) value).bytes(); + } + return ByteBuffers.toByteArray((ByteBuffer) value); + case BINARY: + return ByteBuffers.toByteArray((ByteBuffer) value); + case STRUCT: + Types.StructType structType = (Types.StructType) type; + + if (structType.fields().isEmpty()) { + return new GenericInternalRow(); + } + + List fields = structType.fields(); + Object[] values = new Object[fields.size()]; + StructLike struct = (StructLike) value; + + for (int index = 0; index < fields.size(); index++) { + Types.NestedField field = fields.get(index); + Type fieldType = field.type(); + values[index] = + convertConstant(fieldType, struct.get(index, fieldType.typeId().javaClass())); + } + + return new GenericInternalRow(values); + default: + } + + return value; + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java index af16d9bbc290..65e5843e39b3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java @@ -44,6 +44,7 @@ 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.Maps; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Type.TypeID; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.UUIDUtil; @@ -165,6 +166,7 @@ public ParquetValueReader struct( int defaultMaxDefinitionLevel = type.getMaxDefinitionLevel(currentPath()); for (Types.NestedField field : expectedFields) { int id = field.fieldId(); + ParquetValueReader reader = readersById.get(id); if (idToConstant.containsKey(id)) { // containsKey is used because the constant may be null int fieldMaxDefinitionLevel = @@ -178,15 +180,21 @@ public ParquetValueReader struct( } else if (id == MetadataColumns.IS_DELETED.fieldId()) { reorderedFields.add(ParquetValueReaders.constant(false)); types.add(null); + } else if (reader != null) { + reorderedFields.add(reader); + types.add(typesById.get(id)); + } else if (field.initialDefault() != null) { + reorderedFields.add( + ParquetValueReaders.constant( + SparkUtil.convertConstant(field.type(), field.initialDefault()), + maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel))); + types.add(typesById.get(id)); + } else if (field.isOptional()) { + reorderedFields.add(ParquetValueReaders.nulls()); + types.add(null); } else { - ParquetValueReader reader = readersById.get(id); - if (reader != null) { - reorderedFields.add(reader); - types.add(typesById.get(id)); - } else { - reorderedFields.add(ParquetValueReaders.nulls()); - types.add(null); - } + throw new IllegalArgumentException( + String.format("Missing required field: %s", field.name())); } } @@ -250,7 +258,7 @@ public ParquetValueReader primitive( if (expected != null && expected.typeId() == Types.LongType.get().typeId()) { return new IntAsLongReader(desc); } else { - return new UnboxedReader(desc); + return new UnboxedReader<>(desc); } case DATE: case INT_64: diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index f8e8a1f1dd6b..6f0ee1d2e2a0 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -20,8 +20,6 @@ import java.io.Closeable; import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -29,8 +27,6 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.avro.generic.GenericData; -import org.apache.avro.util.Utf8; import org.apache.iceberg.ContentFile; import org.apache.iceberg.ContentScanTask; import org.apache.iceberg.DeleteFile; @@ -53,16 +49,11 @@ import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.spark.SparkExecutorCache; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Types.StructType; -import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.PartitionUtil; import org.apache.spark.rdd.InputFileBlockHolder; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.unsafe.types.UTF8String; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -193,59 +184,12 @@ private Map inputFiles() { protected Map constantsMap(ContentScanTask task, Schema readSchema) { if (readSchema.findField(MetadataColumns.PARTITION_COLUMN_ID) != null) { StructType partitionType = Partitioning.partitionType(table); - return PartitionUtil.constantsMap(task, partitionType, BaseReader::convertConstant); + return PartitionUtil.constantsMap(task, partitionType, SparkUtil::convertConstant); } else { - return PartitionUtil.constantsMap(task, BaseReader::convertConstant); + return PartitionUtil.constantsMap(task, SparkUtil::convertConstant); } } - protected static Object convertConstant(Type type, Object value) { - if (value == null) { - return null; - } - - switch (type.typeId()) { - case DECIMAL: - return Decimal.apply((BigDecimal) value); - case STRING: - if (value instanceof Utf8) { - Utf8 utf8 = (Utf8) value; - return UTF8String.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); - } - return UTF8String.fromString(value.toString()); - case FIXED: - if (value instanceof byte[]) { - return value; - } else if (value instanceof GenericData.Fixed) { - return ((GenericData.Fixed) value).bytes(); - } - return ByteBuffers.toByteArray((ByteBuffer) value); - case BINARY: - return ByteBuffers.toByteArray((ByteBuffer) value); - case STRUCT: - StructType structType = (StructType) type; - - if (structType.fields().isEmpty()) { - return new GenericInternalRow(); - } - - List fields = structType.fields(); - Object[] values = new Object[fields.size()]; - StructLike struct = (StructLike) value; - - for (int index = 0; index < fields.size(); index++) { - NestedField field = fields.get(index); - Type fieldType = field.type(); - values[index] = - convertConstant(fieldType, struct.get(index, fieldType.typeId().javaClass())); - } - - return new GenericInternalRow(values); - default: - } - return value; - } - protected class SparkDeleteFilter extends DeleteFilter { private final InternalRowWrapper asStructLike; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index f9f24834546f..5511ce24337e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -42,6 +42,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; +import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData.Record; import org.apache.iceberg.DataFile; @@ -246,7 +247,7 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); assertThat(actual).as("Should be a Seq").isInstanceOf(Seq.class); List asList = seqAsJavaListConverter((Seq) actual).asJava(); - assertEqualsSafe(type.asNestedType().asListType(), (Collection) expected, asList); + assertEqualsSafe(type.asNestedType().asListType(), (Collection) expected, asList); break; case MAP: assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); @@ -263,11 +264,20 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) public static void assertEqualsUnsafe(Types.StructType struct, Record rec, InternalRow row) { List fields = struct.fields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i).type(); - - Object expectedValue = rec.get(i); - Object actualValue = row.isNullAt(i) ? null : row.get(i, convert(fieldType)); + for (int readPos = 0; readPos < fields.size(); readPos += 1) { + Types.NestedField field = fields.get(readPos); + Field writeField = rec.getSchema().getField(field.name()); + + Type fieldType = field.type(); + Object actualValue = row.isNullAt(readPos) ? null : row.get(readPos, convert(fieldType)); + + Object expectedValue; + if (writeField != null) { + int writePos = writeField.pos(); + expectedValue = rec.get(writePos); + } else { + expectedValue = field.initialDefault(); + } assertEqualsUnsafe(fieldType, expectedValue, actualValue); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java index ab0d45c3b7ca..0ac0bb530c77 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; @@ -63,32 +64,36 @@ public class TestSparkParquetReader extends AvroDataTest { @Override protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(schema, schema); + } + + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { assumeThat( TypeUtil.find( - schema, + writeSchema, type -> type.isMapType() && type.asMapType().keyType() != Types.StringType.get())) .as("Parquet Avro cannot write non-string map keys") .isNull(); - List expected = RandomData.generateList(schema, 100, 0L); + List expected = RandomData.generateList(writeSchema, 100, 0L); File testFile = File.createTempFile("junit", null, temp.toFile()); assertThat(testFile.delete()).as("Delete should succeed").isTrue(); try (FileAppender writer = - Parquet.write(Files.localOutput(testFile)).schema(schema).named("test").build()) { + Parquet.write(Files.localOutput(testFile)).schema(writeSchema).named("test").build()) { writer.addAll(expected); } try (CloseableIterable reader = Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(type -> SparkParquetReaders.buildReader(schema, type)) + .project(expectedSchema) + .createReaderFunc(type -> SparkParquetReaders.buildReader(expectedSchema, type)) .build()) { Iterator rows = reader.iterator(); for (GenericData.Record record : expected) { assertThat(rows).as("Should have expected number of rows").hasNext(); - assertEqualsUnsafe(schema.asStruct(), record, rows.next()); + assertEqualsUnsafe(expectedSchema.asStruct(), record, rows.next()); } assertThat(rows).as("Should not have extra rows").isExhausted(); } @@ -202,4 +207,220 @@ protected WriteSupport getWriteSupport(Configuration configuration) return new org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport(); } } + + @Test + public void testMissingRequiredWithoutDefault() { + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withDoc("Missing required field with no default") + .build()); + + assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Missing required field: missing_str"); + } + + @Test + public void testDefaultValues() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withInitialDefault("orange") + .build(), + Types.NestedField.optional("missing_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNullDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("missing_date") + .withId(3) + .ofType(Types.DateType.get()) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNestedDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType(Types.StructType.of(required(4, "inner", Types.StringType.get()))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType( + Types.StructType.of( + required(4, "inner", Types.StringType.get()), + Types.NestedField.optional("missing_inner_float") + .withId(5) + .ofType(Types.FloatType.get()) + .withInitialDefault(-0.0F) + .build())) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testMapNestedDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of(required(6, "value_str", Types.StringType.get())))) + .withDoc("Used to test nested map value field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of( + required(6, "value_str", Types.StringType.get()), + Types.NestedField.optional("value_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testListNestedDefaultValue() throws IOException { + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, Types.StructType.of(required(5, "element_str", Types.StringType.get())))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, + Types.StructType.of( + required(5, "element_str", Types.StringType.get()), + Types.NestedField.optional("element_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } } From d0effc6d73c3187983105b5df14d889ff0283f09 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 18 Dec 2024 13:38:59 -0800 Subject: [PATCH 273/313] Data: Fix Parquet and Avro defaults date/time representation (#11811) --- .../org/apache/iceberg/avro/ValueReaders.java | 25 +- .../apache/iceberg/data/GenericDataUtil.java | 60 ++++ .../iceberg/data/avro/PlannedDataReader.java | 4 +- .../org/apache/iceberg/data/DataTest.java | 292 ++++++++++++++++++ .../apache/iceberg/data/DataTestHelpers.java | 5 +- .../iceberg/data/avro/TestGenericData.java | 222 +------------ .../iceberg/data/parquet/TestGenericData.java | 222 +------------ .../data/parquet/BaseParquetReaders.java | 6 +- .../data/parquet/GenericParquetReaders.java | 6 + .../org/apache/iceberg/spark/SparkUtil.java | 5 +- .../spark/data/SparkParquetReaders.java | 2 +- .../spark/data/SparkPlannedAvroReader.java | 4 +- .../iceberg/spark/source/BaseReader.java | 4 +- .../iceberg/spark/data/AvroDataTest.java | 292 ++++++++++++++++++ .../iceberg/spark/data/TestHelpers.java | 23 +- .../spark/data/TestSparkAvroReader.java | 18 +- .../spark/data/TestSparkParquetReader.java | 204 +----------- 17 files changed, 742 insertions(+), 652 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/data/GenericDataUtil.java diff --git a/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java index 67f53d3636a6..66cc2495b016 100644 --- a/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java @@ -33,6 +33,7 @@ import java.util.Map; import java.util.Objects; import java.util.UUID; +import java.util.function.BiFunction; import java.util.function.Supplier; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; @@ -44,6 +45,7 @@ import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.UUIDUtil; @@ -199,6 +201,25 @@ public static List>> buildReadPlan( Schema record, List> fieldReaders, Map idToConstant) { + return buildReadPlan(expected, record, fieldReaders, idToConstant, (type, value) -> value); + } + + /** + * Builds a read plan for record classes that use planned reads instead of a ResolvingDecoder. + * + * @param expected expected StructType + * @param record Avro record schema + * @param fieldReaders list of readers for each field in the Avro record schema + * @param idToConstant a map of field ID to constants values + * @param convert function to convert from internal classes to the target object model + * @return a read plan that is a list of (position, reader) pairs + */ + public static List>> buildReadPlan( + Types.StructType expected, + Schema record, + List> fieldReaders, + Map idToConstant, + BiFunction convert) { Map idToPos = idToPos(expected); List>> readPlan = Lists.newArrayList(); @@ -228,7 +249,9 @@ public static List>> buildReadPlan( if (constant != null) { readPlan.add(Pair.of(pos, ValueReaders.constant(constant))); } else if (field.initialDefault() != null) { - readPlan.add(Pair.of(pos, ValueReaders.constant(field.initialDefault()))); + readPlan.add( + Pair.of( + pos, ValueReaders.constant(convert.apply(field.type(), field.initialDefault())))); } else if (fieldId == MetadataColumns.IS_DELETED.fieldId()) { readPlan.add(Pair.of(pos, ValueReaders.constant(false))); } else if (fieldId == MetadataColumns.ROW_POSITION.fieldId()) { diff --git a/core/src/main/java/org/apache/iceberg/data/GenericDataUtil.java b/core/src/main/java/org/apache/iceberg/data/GenericDataUtil.java new file mode 100644 index 000000000000..152ef31ac876 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/data/GenericDataUtil.java @@ -0,0 +1,60 @@ +/* + * 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.data; + +import java.nio.ByteBuffer; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.DateTimeUtil; + +/** Utility methods for working with Iceberg's generic data model */ +public class GenericDataUtil { + private GenericDataUtil() {} + + /** + * Convert a value from Iceberg's internal data model to the generic data model. + * + * @param type a data type + * @param value value to convert + * @return the value in the generic data model representation + */ + public static Object internalToGeneric(Type type, Object value) { + if (null == value) { + return null; + } + + switch (type.typeId()) { + case DATE: + return DateTimeUtil.dateFromDays((Integer) value); + case TIME: + return DateTimeUtil.timeFromMicros((Long) value); + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + return DateTimeUtil.timestamptzFromMicros((Long) value); + } else { + return DateTimeUtil.timestampFromMicros((Long) value); + } + case FIXED: + return ByteBuffers.toByteArray((ByteBuffer) value); + } + + return value; + } +} diff --git a/core/src/main/java/org/apache/iceberg/data/avro/PlannedDataReader.java b/core/src/main/java/org/apache/iceberg/data/avro/PlannedDataReader.java index c7ec2e6091cc..64b3e943e270 100644 --- a/core/src/main/java/org/apache/iceberg/data/avro/PlannedDataReader.java +++ b/core/src/main/java/org/apache/iceberg/data/avro/PlannedDataReader.java @@ -32,6 +32,7 @@ import org.apache.iceberg.avro.SupportsRowPosition; import org.apache.iceberg.avro.ValueReader; import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.data.GenericDataUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -97,7 +98,8 @@ public ValueReader record(Type partner, Schema record, List> f Types.StructType expected = partner.asStructType(); List>> readPlan = - ValueReaders.buildReadPlan(expected, record, fieldReaders, idToConstant); + ValueReaders.buildReadPlan( + expected, record, fieldReaders, idToConstant, GenericDataUtil::internalToGeneric); return GenericReaders.struct(readPlan, expected); } diff --git a/data/src/test/java/org/apache/iceberg/data/DataTest.java b/data/src/test/java/org/apache/iceberg/data/DataTest.java index 638a344cd2bc..657fa805e5a6 100644 --- a/data/src/test/java/org/apache/iceberg/data/DataTest.java +++ b/data/src/test/java/org/apache/iceberg/data/DataTest.java @@ -20,24 +20,45 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.nio.file.Path; +import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.ListType; import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.types.Types.MapType; import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.util.DateTimeUtil; +import org.assertj.core.api.Assumptions; 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.Arguments; +import org.junit.jupiter.params.provider.MethodSource; public abstract class DataTest { protected abstract void writeAndValidate(Schema schema) throws IOException; + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + throw new UnsupportedEncodingException( + "Cannot run test, writeAndValidate(Schema, Schema) is not implemented"); + } + + protected boolean supportsDefaultValues() { + return false; + } + protected static final StructType SUPPORTED_PRIMITIVES = StructType.of( required(100, "id", LongType.get()), @@ -194,4 +215,275 @@ public void testMixedTypes() throws IOException { writeAndValidate(schema); } + + @Test + public void testMissingRequiredWithoutDefault() { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withDoc("Missing required field with no default") + .build()); + + assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Missing required field: missing_str"); + } + + @Test + public void testDefaultValues() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withInitialDefault("orange") + .build(), + Types.NestedField.optional("missing_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNullDefaultValue() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("missing_date") + .withId(3) + .ofType(Types.DateType.get()) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNestedDefaultValue() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType(Types.StructType.of(required(4, "inner", Types.StringType.get()))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType( + Types.StructType.of( + required(4, "inner", Types.StringType.get()), + Types.NestedField.optional("missing_inner_float") + .withId(5) + .ofType(Types.FloatType.get()) + .withInitialDefault(-0.0F) + .build())) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testMapNestedDefaultValue() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of(required(6, "value_str", Types.StringType.get())))) + .withDoc("Used to test nested map value field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of( + required(6, "value_str", Types.StringType.get()), + Types.NestedField.optional("value_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testListNestedDefaultValue() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, Types.StructType.of(required(5, "element_str", Types.StringType.get())))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, + Types.StructType.of( + required(5, "element_str", Types.StringType.get()), + Types.NestedField.optional("element_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + private static Stream primitiveTypesAndDefaults() { + return Stream.of( + Arguments.of(Types.BooleanType.get(), false), + Arguments.of(Types.IntegerType.get(), 34), + Arguments.of(Types.LongType.get(), 4900000000L), + Arguments.of(Types.FloatType.get(), 12.21F), + Arguments.of(Types.DoubleType.get(), -0.0D), + Arguments.of(Types.DateType.get(), DateTimeUtil.isoDateToDays("2024-12-17")), + Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), + Arguments.of( + Types.TimestampType.withZone(), + DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00")), + Arguments.of( + Types.TimestampType.withoutZone(), + DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999")), + Arguments.of(Types.StringType.get(), "iceberg"), + Arguments.of(Types.UUIDType.get(), UUID.randomUUID()), + Arguments.of( + Types.FixedType.ofLength(4), ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), + Arguments.of(Types.BinaryType.get(), ByteBuffer.wrap(new byte[] {0x0a, 0x0b})), + Arguments.of(Types.DecimalType.of(9, 2), new BigDecimal("12.34"))); + } + + @ParameterizedTest + @MethodSource("primitiveTypesAndDefaults") + public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defaultValue) + throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema readSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("col_with_default") + .withId(2) + .ofType(type) + .withInitialDefault(defaultValue) + .build()); + + writeAndValidate(writeSchema, readSchema); + } } diff --git a/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java index 72e3973382af..f8536dfd01c5 100644 --- a/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java +++ b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java @@ -36,7 +36,10 @@ public static void assertEquals(Types.StructType struct, Record expected, Record assertEquals( field.type(), expected.getField(expectedField.name()), actual.getField(field.name())); } else { - assertThat(actual.getField(field.name())).isEqualTo(field.initialDefault()); + assertEquals( + field.type(), + GenericDataUtil.internalToGeneric(field.type(), field.initialDefault()), + actual.getField(field.name())); } } } diff --git a/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java index 651df22cfc15..bf5160fd18dc 100644 --- a/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java +++ b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java @@ -18,9 +18,7 @@ */ package org.apache.iceberg.data.avro; -import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -35,8 +33,6 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; public class TestGenericData extends DataTest { @Override @@ -44,6 +40,7 @@ protected void writeAndValidate(Schema schema) throws IOException { writeAndValidate(schema, schema); } + @Override protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { List expected = RandomGenericData.generate(writeSchema, 100, 0L); @@ -75,219 +72,8 @@ protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throw } } - @Test - public void testMissingRequiredWithoutDefault() { - Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.required("missing_str") - .withId(6) - .ofType(Types.StringType.get()) - .withDoc("Missing required field with no default") - .build()); - - assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Missing required field: missing_str"); - } - - @Test - public void testDefaultValues() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.required("missing_str") - .withId(6) - .ofType(Types.StringType.get()) - .withInitialDefault("orange") - .build(), - Types.NestedField.optional("missing_int") - .withId(7) - .ofType(Types.IntegerType.get()) - .withInitialDefault(34) - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testNullDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.optional("missing_date") - .withId(3) - .ofType(Types.DateType.get()) - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testNestedDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build(), - Types.NestedField.optional("nested") - .withId(3) - .ofType(Types.StructType.of(required(4, "inner", Types.StringType.get()))) - .withDoc("Used to test nested field defaults") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.optional("nested") - .withId(3) - .ofType( - Types.StructType.of( - required(4, "inner", Types.StringType.get()), - Types.NestedField.optional("missing_inner_float") - .withId(5) - .ofType(Types.FloatType.get()) - .withInitialDefault(-0.0F) - .build())) - .withDoc("Used to test nested field defaults") - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testMapNestedDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build(), - Types.NestedField.optional("nested_map") - .withId(3) - .ofType( - Types.MapType.ofOptional( - 4, - 5, - Types.StringType.get(), - Types.StructType.of(required(6, "value_str", Types.StringType.get())))) - .withDoc("Used to test nested map value field defaults") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.optional("nested_map") - .withId(3) - .ofType( - Types.MapType.ofOptional( - 4, - 5, - Types.StringType.get(), - Types.StructType.of( - required(6, "value_str", Types.StringType.get()), - Types.NestedField.optional("value_int") - .withId(7) - .ofType(Types.IntegerType.get()) - .withInitialDefault(34) - .build()))) - .withDoc("Used to test nested field defaults") - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testListNestedDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build(), - Types.NestedField.optional("nested_list") - .withId(3) - .ofType( - Types.ListType.ofOptional( - 4, Types.StructType.of(required(5, "element_str", Types.StringType.get())))) - .withDoc("Used to test nested field defaults") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.optional("nested_list") - .withId(3) - .ofType( - Types.ListType.ofOptional( - 4, - Types.StructType.of( - required(5, "element_str", Types.StringType.get()), - Types.NestedField.optional("element_int") - .withId(7) - .ofType(Types.IntegerType.get()) - .withInitialDefault(34) - .build()))) - .withDoc("Used to test nested field defaults") - .build()); - - writeAndValidate(writeSchema, expectedSchema); + @Override + protected boolean supportsDefaultValues() { + return true; } } diff --git a/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java index 5c7c11f1d231..5a63f7a1fc9d 100644 --- a/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java +++ b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java @@ -19,9 +19,7 @@ package org.apache.iceberg.data.parquet; import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -43,7 +41,6 @@ import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.apache.iceberg.types.Types.NestedField; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; import org.junit.jupiter.api.Test; @@ -54,6 +51,7 @@ protected void writeAndValidate(Schema schema) throws IOException { writeAndValidate(schema, schema); } + @Override protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { List expected = RandomGenericData.generate(writeSchema, 100, 12228L); @@ -98,6 +96,11 @@ protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throw } } + @Override + protected boolean supportsDefaultValues() { + return true; + } + @Test public void testTwoLevelList() throws IOException { Schema schema = @@ -144,217 +147,4 @@ public void testTwoLevelList() throws IOException { assertThat(Lists.newArrayList(reader).size()).isEqualTo(1); } } - - @Test - public void testMissingRequiredWithoutDefault() { - Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.required("missing_str") - .withId(6) - .ofType(Types.StringType.get()) - .withDoc("Missing required field with no default") - .build()); - - assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Missing required field: missing_str"); - } - - @Test - public void testDefaultValues() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - NestedField.required("missing_str") - .withId(6) - .ofType(Types.StringType.get()) - .withInitialDefault("orange") - .build(), - NestedField.optional("missing_int") - .withId(7) - .ofType(Types.IntegerType.get()) - .withInitialDefault(34) - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testNullDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - NestedField.optional("missing_date").withId(3).ofType(Types.DateType.get()).build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testNestedDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build(), - NestedField.optional("nested") - .withId(3) - .ofType(Types.StructType.of(required(4, "inner", Types.StringType.get()))) - .withDoc("Used to test nested field defaults") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - NestedField.optional("nested") - .withId(3) - .ofType( - Types.StructType.of( - required(4, "inner", Types.StringType.get()), - NestedField.optional("missing_inner_float") - .withId(5) - .ofType(Types.FloatType.get()) - .withInitialDefault(-0.0F) - .build())) - .withDoc("Used to test nested field defaults") - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testMapNestedDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build(), - NestedField.optional("nested_map") - .withId(3) - .ofType( - Types.MapType.ofOptional( - 4, - 5, - Types.StringType.get(), - Types.StructType.of(required(6, "value_str", Types.StringType.get())))) - .withDoc("Used to test nested map value field defaults") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - NestedField.optional("nested_map") - .withId(3) - .ofType( - Types.MapType.ofOptional( - 4, - 5, - Types.StringType.get(), - Types.StructType.of( - required(6, "value_str", Types.StringType.get()), - Types.NestedField.optional("value_int") - .withId(7) - .ofType(Types.IntegerType.get()) - .withInitialDefault(34) - .build()))) - .withDoc("Used to test nested field defaults") - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testListNestedDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build(), - NestedField.optional("nested_list") - .withId(3) - .ofType( - Types.ListType.ofOptional( - 4, Types.StructType.of(required(5, "element_str", Types.StringType.get())))) - .withDoc("Used to test nested field defaults") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - NestedField.optional("nested_list") - .withId(3) - .ofType( - Types.ListType.ofOptional( - 4, - Types.StructType.of( - required(5, "element_str", Types.StringType.get()), - Types.NestedField.optional("element_int") - .withId(7) - .ofType(Types.IntegerType.get()) - .withInitialDefault(34) - .build()))) - .withDoc("Used to test nested field defaults") - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } } diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java index b3a1948d0a35..efdf9cc9b01d 100644 --- a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java @@ -76,6 +76,10 @@ protected ParquetValueReader createReader( protected abstract ParquetValueReader createStructReader( List types, List> fieldReaders, Types.StructType structType); + protected Object convertConstant(org.apache.iceberg.types.Type type, Object value) { + return value; + } + private class FallbackReadBuilder extends ReadBuilder { private FallbackReadBuilder(MessageType type, Map idToConstant) { super(type, idToConstant); @@ -283,7 +287,7 @@ public ParquetValueReader struct( } else if (field.initialDefault() != null) { reorderedFields.add( ParquetValueReaders.constant( - field.initialDefault(), + convertConstant(field.type(), field.initialDefault()), maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel))); types.add(typesById.get(id)); } else if (field.isOptional()) { diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java b/parquet/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java index 99b8c9baad64..8023cef71dae 100644 --- a/parquet/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericDataUtil; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.parquet.ParquetValueReader; @@ -51,6 +52,11 @@ protected ParquetValueReader createStructReader( return new RecordReader(types, fieldReaders, structType); } + @Override + protected Object convertConstant(org.apache.iceberg.types.Type type, Object value) { + return GenericDataUtil.internalToGeneric(type, value); + } + private static class RecordReader extends StructReader { private final GenericRecord template; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java index 4bd2e9c21551..c88a907e0f29 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java @@ -287,7 +287,7 @@ private static String toExecutorLocation(BlockManagerId id) { * @param value a value that is an instance of {@link Type.TypeID#javaClass()} * @return the value converted for Spark */ - public static Object convertConstant(Type type, Object value) { + public static Object internalToSpark(Type type, Object value) { if (value == null) { return null; } @@ -295,6 +295,7 @@ public static Object convertConstant(Type type, Object value) { switch (type.typeId()) { case DECIMAL: return Decimal.apply((BigDecimal) value); + case UUID: case STRING: if (value instanceof Utf8) { Utf8 utf8 = (Utf8) value; @@ -325,7 +326,7 @@ public static Object convertConstant(Type type, Object value) { Types.NestedField field = fields.get(index); Type fieldType = field.type(); values[index] = - convertConstant(fieldType, struct.get(index, fieldType.typeId().javaClass())); + internalToSpark(fieldType, struct.get(index, fieldType.typeId().javaClass())); } return new GenericInternalRow(values); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java index 65e5843e39b3..3ce54d2d9ffa 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java @@ -186,7 +186,7 @@ public ParquetValueReader struct( } else if (field.initialDefault() != null) { reorderedFields.add( ParquetValueReaders.constant( - SparkUtil.convertConstant(field.type(), field.initialDefault()), + SparkUtil.internalToSpark(field.type(), field.initialDefault()), maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel))); types.add(typesById.get(id)); } else if (field.isOptional()) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java index dc4af24685b3..7bcd8881c10b 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java @@ -32,6 +32,7 @@ import org.apache.iceberg.avro.ValueReader; import org.apache.iceberg.avro.ValueReaders; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; @@ -97,7 +98,8 @@ public ValueReader record(Type partner, Schema record, List> f Types.StructType expected = partner.asStructType(); List>> readPlan = - ValueReaders.buildReadPlan(expected, record, fieldReaders, idToConstant); + ValueReaders.buildReadPlan( + expected, record, fieldReaders, idToConstant, SparkUtil::internalToSpark); // TODO: should this pass expected so that struct.get can reuse containers? return SparkValueReaders.struct(readPlan, expected.fields().size()); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index 6f0ee1d2e2a0..6b3c3d3f2cf3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -184,9 +184,9 @@ private Map inputFiles() { protected Map constantsMap(ContentScanTask task, Schema readSchema) { if (readSchema.findField(MetadataColumns.PARTITION_COLUMN_ID) != null) { StructType partitionType = Partitioning.partitionType(table); - return PartitionUtil.constantsMap(task, partitionType, SparkUtil::convertConstant); + return PartitionUtil.constantsMap(task, partitionType, SparkUtil::internalToSpark); } else { - return PartitionUtil.constantsMap(task, SparkUtil::convertConstant); + return PartitionUtil.constantsMap(task, SparkUtil::internalToSpark); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java index 8f90a51a6e30..4f7eab30a47d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java @@ -20,28 +20,49 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.nio.file.Path; import java.util.Map; +import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; import org.apache.iceberg.Schema; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.ListType; import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.types.Types.MapType; import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.internal.SQLConf; +import org.assertj.core.api.Assumptions; 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.Arguments; +import org.junit.jupiter.params.provider.MethodSource; public abstract class AvroDataTest { protected abstract void writeAndValidate(Schema schema) throws IOException; + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + throw new UnsupportedEncodingException( + "Cannot run test, writeAndValidate(Schema, Schema) is not implemented"); + } + + protected boolean supportsDefaultValues() { + return false; + } + protected static final StructType SUPPORTED_PRIMITIVES = StructType.of( required(100, "id", LongType.get()), @@ -238,6 +259,277 @@ public void testTimestampWithoutZone() throws IOException { writeAndValidate(schema); } + @Test + public void testMissingRequiredWithoutDefault() { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withDoc("Missing required field with no default") + .build()); + + assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Missing required field: missing_str"); + } + + @Test + public void testDefaultValues() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withInitialDefault("orange") + .build(), + Types.NestedField.optional("missing_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNullDefaultValue() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("missing_date") + .withId(3) + .ofType(Types.DateType.get()) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNestedDefaultValue() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType(Types.StructType.of(required(4, "inner", Types.StringType.get()))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType( + Types.StructType.of( + required(4, "inner", Types.StringType.get()), + Types.NestedField.optional("missing_inner_float") + .withId(5) + .ofType(Types.FloatType.get()) + .withInitialDefault(-0.0F) + .build())) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testMapNestedDefaultValue() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of(required(6, "value_str", Types.StringType.get())))) + .withDoc("Used to test nested map value field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of( + required(6, "value_str", Types.StringType.get()), + Types.NestedField.optional("value_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testListNestedDefaultValue() throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, Types.StructType.of(required(5, "element_str", Types.StringType.get())))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault("wrong!") + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, + Types.StructType.of( + required(5, "element_str", Types.StringType.get()), + Types.NestedField.optional("element_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(34) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + private static Stream primitiveTypesAndDefaults() { + return Stream.of( + Arguments.of(Types.BooleanType.get(), false), + Arguments.of(Types.IntegerType.get(), 34), + Arguments.of(Types.LongType.get(), 4900000000L), + Arguments.of(Types.FloatType.get(), 12.21F), + Arguments.of(Types.DoubleType.get(), -0.0D), + Arguments.of(Types.DateType.get(), DateTimeUtil.isoDateToDays("2024-12-17")), + // Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), + Arguments.of( + Types.TimestampType.withZone(), + DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00")), + Arguments.of( + Types.TimestampType.withoutZone(), + DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999")), + Arguments.of(Types.StringType.get(), "iceberg"), + Arguments.of(Types.UUIDType.get(), UUID.randomUUID()), + Arguments.of( + Types.FixedType.ofLength(4), ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), + Arguments.of(Types.BinaryType.get(), ByteBuffer.wrap(new byte[] {0x0a, 0x0b})), + Arguments.of(Types.DecimalType.of(9, 2), new BigDecimal("12.34"))); + } + + @ParameterizedTest + @MethodSource("primitiveTypesAndDefaults") + public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defaultValue) + throws IOException { + Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema readSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("col_with_default") + .withId(2) + .ofType(type) + .withInitialDefault(defaultValue) + .build()); + + writeAndValidate(writeSchema, readSchema); + } + protected void withSQLConf(Map conf, Action action) throws IOException { SQLConf sqlConf = SQLConf.get(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 5511ce24337e..d3d69e4b9d86 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -60,6 +60,7 @@ import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.DeleteFileSet; import org.apache.orc.storage.serde2.io.DateWritable; import org.apache.spark.sql.Column; @@ -356,11 +357,21 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual .isEqualTo(String.valueOf(expected)); break; case FIXED: - assertThat(expected).as("Should expect a Fixed").isInstanceOf(GenericData.Fixed.class); + // generated data is written using Avro or Parquet/Avro so generated rows use + // GenericData.Fixed, but default values are converted from Iceberg's internal + // representation so the expected value may be either class. + byte[] expectedBytes; + if (expected instanceof ByteBuffer) { + expectedBytes = ByteBuffers.toByteArray((ByteBuffer) expected); + } else if (expected instanceof GenericData.Fixed) { + expectedBytes = ((GenericData.Fixed) expected).bytes(); + } else { + throw new IllegalStateException( + "Invalid expected value, not byte[] or Fixed: " + expected); + } + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); - assertThat(actual) - .as("Bytes should match") - .isEqualTo(((GenericData.Fixed) expected).bytes()); + assertThat(actual).as("Bytes should match").isEqualTo(expectedBytes); break; case BINARY: assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); @@ -384,12 +395,12 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); assertThat(actual).as("Should be an ArrayData").isInstanceOf(ArrayData.class); assertEqualsUnsafe( - type.asNestedType().asListType(), (Collection) expected, (ArrayData) actual); + type.asNestedType().asListType(), (Collection) expected, (ArrayData) actual); break; case MAP: assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); assertThat(actual).as("Should be an ArrayBasedMapData").isInstanceOf(MapData.class); - assertEqualsUnsafe(type.asNestedType().asMapType(), (Map) expected, (MapData) actual); + assertEqualsUnsafe(type.asNestedType().asMapType(), (Map) expected, (MapData) actual); break; case TIME: default: diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java index 7f9bcbacf298..002539a97620 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java @@ -36,13 +36,18 @@ public class TestSparkAvroReader extends AvroDataTest { @Override protected void writeAndValidate(Schema schema) throws IOException { - List expected = RandomData.generateList(schema, 100, 0L); + writeAndValidate(schema, schema); + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + List expected = RandomData.generateList(writeSchema, 100, 0L); File testFile = File.createTempFile("junit", null, temp.toFile()); assertThat(testFile.delete()).as("Delete should succeed").isTrue(); try (FileAppender writer = - Avro.write(Files.localOutput(testFile)).schema(schema).named("test").build()) { + Avro.write(Files.localOutput(testFile)).schema(writeSchema).named("test").build()) { for (Record rec : expected) { writer.add(rec); } @@ -52,13 +57,18 @@ protected void writeAndValidate(Schema schema) throws IOException { try (AvroIterable reader = Avro.read(Files.localInput(testFile)) .createResolvingReader(SparkPlannedAvroReader::create) - .project(schema) + .project(expectedSchema) .build()) { rows = Lists.newArrayList(reader); } for (int i = 0; i < expected.size(); i += 1) { - assertEqualsUnsafe(schema.asStruct(), expected.get(i), rows.get(i)); + assertEqualsUnsafe(expectedSchema.asStruct(), expected.get(i), rows.get(i)); } } + + @Override + protected boolean supportsDefaultValues() { + return true; + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java index 0ac0bb530c77..1cd4fccfdd3f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java @@ -67,6 +67,7 @@ protected void writeAndValidate(Schema schema) throws IOException { writeAndValidate(schema, schema); } + @Override protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { assumeThat( TypeUtil.find( @@ -99,6 +100,11 @@ protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throw } } + @Override + protected boolean supportsDefaultValues() { + return true; + } + protected List rowsFromFile(InputFile inputFile, Schema schema) throws IOException { try (CloseableIterable reader = Parquet.read(inputFile) @@ -225,202 +231,4 @@ public void testMissingRequiredWithoutDefault() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Missing required field: missing_str"); } - - @Test - public void testDefaultValues() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.required("missing_str") - .withId(6) - .ofType(Types.StringType.get()) - .withInitialDefault("orange") - .build(), - Types.NestedField.optional("missing_int") - .withId(7) - .ofType(Types.IntegerType.get()) - .withInitialDefault(34) - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testNullDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.optional("missing_date") - .withId(3) - .ofType(Types.DateType.get()) - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testNestedDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build(), - Types.NestedField.optional("nested") - .withId(3) - .ofType(Types.StructType.of(required(4, "inner", Types.StringType.get()))) - .withDoc("Used to test nested field defaults") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.optional("nested") - .withId(3) - .ofType( - Types.StructType.of( - required(4, "inner", Types.StringType.get()), - Types.NestedField.optional("missing_inner_float") - .withId(5) - .ofType(Types.FloatType.get()) - .withInitialDefault(-0.0F) - .build())) - .withDoc("Used to test nested field defaults") - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testMapNestedDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build(), - Types.NestedField.optional("nested_map") - .withId(3) - .ofType( - Types.MapType.ofOptional( - 4, - 5, - Types.StringType.get(), - Types.StructType.of(required(6, "value_str", Types.StringType.get())))) - .withDoc("Used to test nested map value field defaults") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.optional("nested_map") - .withId(3) - .ofType( - Types.MapType.ofOptional( - 4, - 5, - Types.StringType.get(), - Types.StructType.of( - required(6, "value_str", Types.StringType.get()), - Types.NestedField.optional("value_int") - .withId(7) - .ofType(Types.IntegerType.get()) - .withInitialDefault(34) - .build()))) - .withDoc("Used to test nested field defaults") - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } - - @Test - public void testListNestedDefaultValue() throws IOException { - Schema writeSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .withDoc("Should not produce default value") - .build(), - Types.NestedField.optional("nested_list") - .withId(3) - .ofType( - Types.ListType.ofOptional( - 4, Types.StructType.of(required(5, "element_str", Types.StringType.get())))) - .withDoc("Used to test nested field defaults") - .build()); - - Schema expectedSchema = - new Schema( - required(1, "id", Types.LongType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withInitialDefault("wrong!") - .build(), - Types.NestedField.optional("nested_list") - .withId(3) - .ofType( - Types.ListType.ofOptional( - 4, - Types.StructType.of( - required(5, "element_str", Types.StringType.get()), - Types.NestedField.optional("element_int") - .withId(7) - .ofType(Types.IntegerType.get()) - .withInitialDefault(34) - .build()))) - .withDoc("Used to test nested field defaults") - .build()); - - writeAndValidate(writeSchema, expectedSchema); - } } From 91a1505d09cebcd1d088ac53cd42732c343883de Mon Sep 17 00:00:00 2001 From: Marc Cenac <547446+mrcnc@users.noreply.github.com> Date: Wed, 18 Dec 2024 16:55:05 -0600 Subject: [PATCH 274/313] Revert "Support WASB scheme in ADLSFileIO (#11504)" (#11812) This reverts commit 09634857e4a1333f5dc742d1dca3921e9a9f62dd. --- .../apache/iceberg/azure/AzureProperties.java | 13 +----------- .../iceberg/azure/adlsv2/ADLSLocation.java | 20 ++++++------------- .../iceberg/azure/AzurePropertiesTest.java | 10 ++++------ .../azure/adlsv2/ADLSLocationTest.java | 19 ++++-------------- .../apache/iceberg/io/ResolvingFileIO.java | 4 +--- 5 files changed, 16 insertions(+), 50 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index a7f9885a4726..2d363cbc5231 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -77,17 +77,6 @@ public Optional adlsWriteBlockSize() { return Optional.ofNullable(adlsWriteBlockSize); } - /** - * Applies configuration to the {@link DataLakeFileSystemClientBuilder} to provide the endpoint - * and credentials required to create an instance of the client. - * - *

    The default endpoint is constructed in the form {@code - * https://{account}.dfs.core.windows.net} and default credentials are provided via the {@link - * com.azure.identity.DefaultAzureCredential}. - * - * @param account the service account name - * @param builder the builder instance - */ public void applyClientConfiguration(String account, DataLakeFileSystemClientBuilder builder) { String sasToken = adlsSasTokens.get(account); if (sasToken != null && !sasToken.isEmpty()) { @@ -104,7 +93,7 @@ public void applyClientConfiguration(String account, DataLakeFileSystemClientBui if (connectionString != null && !connectionString.isEmpty()) { builder.endpoint(connectionString); } else { - builder.endpoint("https://" + account + ".dfs.core.windows.net"); + builder.endpoint("https://" + account); } } } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java index fb91c4cb3233..5af590628fe8 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSLocation.java @@ -30,21 +30,14 @@ * *

    Locations follow a URI like structure to identify resources * - *

    {@code abfs[s]://[@].dfs.core.windows.net/}
    - * - * or - * - *
    {@code wasb[s]://@.blob.core.windows.net/}
    - * - * For compatibility, locations using the wasb scheme are also accepted but will use the Azure Data - * Lake Storage Gen2 REST APIs instead of the Blob Storage REST APIs. + *
    {@code abfs[s]://[@]/}
    * *

    See Azure * Data Lake Storage URI */ class ADLSLocation { - private static final Pattern URI_PATTERN = Pattern.compile("^(abfss?|wasbs?)://([^/?#]+)(.*)?$"); + private static final Pattern URI_PATTERN = Pattern.compile("^abfss?://([^/?#]+)(.*)?$"); private final String storageAccount; private final String container; @@ -62,18 +55,17 @@ class ADLSLocation { ValidationException.check(matcher.matches(), "Invalid ADLS URI: %s", location); - String authority = matcher.group(2); + String authority = matcher.group(1); String[] parts = authority.split("@", -1); if (parts.length > 1) { this.container = parts[0]; - String host = parts[1]; - this.storageAccount = host.split("\\.", -1)[0]; + this.storageAccount = parts[1]; } else { this.container = null; - this.storageAccount = authority.split("\\.", -1)[0]; + this.storageAccount = authority; } - String uriPath = matcher.group(3); + String uriPath = matcher.group(2); this.path = uriPath == null ? "" : uriPath.startsWith("/") ? uriPath.substring(1) : uriPath; } diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index 4f032d7ab125..6b8287c44e58 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -97,13 +97,11 @@ public void testNoSasToken() { @Test public void testWithConnectionString() { AzureProperties props = - new AzureProperties( - ImmutableMap.of( - "adls.connection-string.account1", "https://account1.dfs.core.usgovcloudapi.net")); + new AzureProperties(ImmutableMap.of("adls.connection-string.account1", "http://endpoint")); DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account1", clientBuilder); - verify(clientBuilder).endpoint("https://account1.dfs.core.usgovcloudapi.net"); + verify(clientBuilder).endpoint("http://endpoint"); } @Test @@ -113,7 +111,7 @@ public void testNoMatchingConnectionString() { DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account1", clientBuilder); - verify(clientBuilder).endpoint("https://account1.dfs.core.windows.net"); + verify(clientBuilder).endpoint("https://account1"); } @Test @@ -122,7 +120,7 @@ public void testNoConnectionString() { DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account", clientBuilder); - verify(clientBuilder).endpoint("https://account.dfs.core.windows.net"); + verify(clientBuilder).endpoint("https://account"); } @Test diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java index 10b5e1877cca..403886f4b28e 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/ADLSLocationTest.java @@ -33,18 +33,7 @@ public void testLocationParsing(String scheme) { String p1 = scheme + "://container@account.dfs.core.windows.net/path/to/file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account"); - assertThat(location.container().get()).isEqualTo("container"); - assertThat(location.path()).isEqualTo("path/to/file"); - } - - @ParameterizedTest - @ValueSource(strings = {"wasb", "wasbs"}) - public void testWasbLocatonParsing(String scheme) { - String p1 = scheme + "://container@account.blob.core.windows.net/path/to/file"; - ADLSLocation location = new ADLSLocation(p1); - - assertThat(location.storageAccount()).isEqualTo("account"); + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path/to/file"); } @@ -54,7 +43,7 @@ public void testEncodedString() { String p1 = "abfs://container@account.dfs.core.windows.net/path%20to%20file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account"); + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo("path%20to%20file"); } @@ -78,7 +67,7 @@ public void testNoContainer() { String p1 = "abfs://account.dfs.core.windows.net/path/to/file"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account"); + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().isPresent()).isFalse(); assertThat(location.path()).isEqualTo("path/to/file"); } @@ -88,7 +77,7 @@ public void testNoPath() { String p1 = "abfs://container@account.dfs.core.windows.net"; ADLSLocation location = new ADLSLocation(p1); - assertThat(location.storageAccount()).isEqualTo("account"); + assertThat(location.storageAccount()).isEqualTo("account.dfs.core.windows.net"); assertThat(location.container().get()).isEqualTo("container"); assertThat(location.path()).isEqualTo(""); } diff --git a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java index a8adf979f85a..a858045aab8b 100644 --- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java +++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java @@ -62,9 +62,7 @@ public class ResolvingFileIO implements HadoopConfigurable, DelegateFileIO { "s3n", S3_FILE_IO_IMPL, "gs", GCS_FILE_IO_IMPL, "abfs", ADLS_FILE_IO_IMPL, - "abfss", ADLS_FILE_IO_IMPL, - "wasb", ADLS_FILE_IO_IMPL, - "wasbs", ADLS_FILE_IO_IMPL); + "abfss", ADLS_FILE_IO_IMPL); private final Map ioInstances = Maps.newConcurrentMap(); private final AtomicBoolean isClosed = new AtomicBoolean(false); From 88a25967e72d8137147b50c6cb0a221487050369 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 19 Dec 2024 07:53:41 -0700 Subject: [PATCH 275/313] Core, Spark, Flink, Hive: Remove unused failsafe dependency from core and add Failsafe to runtime LICENSE(s) (#11816) --- build.gradle | 1 - flink/v1.18/flink-runtime/LICENSE | 8 ++++++++ flink/v1.19/flink-runtime/LICENSE | 8 ++++++++ flink/v1.20/flink-runtime/LICENSE | 8 ++++++++ hive-runtime/LICENSE | 8 ++++++++ kafka-connect/kafka-connect-runtime/LICENSE | 6 ++++++ spark/v3.3/spark-runtime/LICENSE | 8 ++++++++ spark/v3.4/spark-runtime/LICENSE | 8 ++++++++ spark/v3.5/spark-runtime/LICENSE | 8 ++++++++ 9 files changed, 62 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index eb63ce138ab2..46fd1e3bb3f9 100644 --- a/build.gradle +++ b/build.gradle @@ -347,7 +347,6 @@ project(':iceberg-core') { implementation libs.jackson.core implementation libs.jackson.databind implementation libs.caffeine - implementation libs.failsafe implementation libs.roaringbitmap compileOnly(libs.hadoop2.client) { exclude group: 'org.apache.avro', module: 'avro' diff --git a/flink/v1.18/flink-runtime/LICENSE b/flink/v1.18/flink-runtime/LICENSE index 8ab53469eb87..bbdabc61e3b6 100644 --- a/flink/v1.18/flink-runtime/LICENSE +++ b/flink/v1.18/flink-runtime/LICENSE @@ -500,3 +500,11 @@ This product includes code from Apache HttpComponents Client. Copyright: 1999-2022 The Apache Software Foundation. Home page: https://hc.apache.org/ License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains failsafe. + +Copyright: Jonathan Halterman and friends +Home page: https://failsafe.dev/ +License: http://www.apache.org/licenses/LICENSE-2.0.html \ No newline at end of file diff --git a/flink/v1.19/flink-runtime/LICENSE b/flink/v1.19/flink-runtime/LICENSE index 8ab53469eb87..e908550aa6a3 100644 --- a/flink/v1.19/flink-runtime/LICENSE +++ b/flink/v1.19/flink-runtime/LICENSE @@ -500,3 +500,11 @@ This product includes code from Apache HttpComponents Client. Copyright: 1999-2022 The Apache Software Foundation. Home page: https://hc.apache.org/ License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains failsafe. + +Copyright: Jonathan Halterman and friends +Home page: https://failsafe.dev/ +License: https://www.apache.org/licenses/LICENSE-2.0.html \ No newline at end of file diff --git a/flink/v1.20/flink-runtime/LICENSE b/flink/v1.20/flink-runtime/LICENSE index 8ab53469eb87..247c814da784 100644 --- a/flink/v1.20/flink-runtime/LICENSE +++ b/flink/v1.20/flink-runtime/LICENSE @@ -500,3 +500,11 @@ This product includes code from Apache HttpComponents Client. Copyright: 1999-2022 The Apache Software Foundation. Home page: https://hc.apache.org/ License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains failsafe. + +Copyright: Jonathan Halterman and friends +Home page: https://failsafe.dev/ +License: https://www.apache.org/licenses/LICENSE-2.0.html diff --git a/hive-runtime/LICENSE b/hive-runtime/LICENSE index 24cd3612e5e4..c40ec91cbd06 100644 --- a/hive-runtime/LICENSE +++ b/hive-runtime/LICENSE @@ -500,3 +500,11 @@ This product includes code from Apache HttpComponents Client. Copyright: 1999-2022 The Apache Software Foundation. Home page: https://hc.apache.org/ License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains failsafe. + +Copyright: Jonathan Halterman and friends +Home page: https://failsafe.dev/ +License: https://www.apache.org/licenses/LICENSE-2.0.html \ No newline at end of file diff --git a/kafka-connect/kafka-connect-runtime/LICENSE b/kafka-connect/kafka-connect-runtime/LICENSE index 6eb4fdee65b8..af0a897ff77b 100644 --- a/kafka-connect/kafka-connect-runtime/LICENSE +++ b/kafka-connect/kafka-connect-runtime/LICENSE @@ -722,6 +722,12 @@ License (from POM): The Apache Software License, Version 2.0 - http://www.apache -------------------------------------------------------------------------------- +Group: dev.failsafe Name: failsafe Version: 3.3.2 +Project URL (from POM): https://github.com/failsafe-lib/failsafe +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + Group: dnsjava Name: dnsjava Version: 2.1.7 Project URL (from POM): http://www.dnsjava.org License (from POM): BSD 2-Clause license - http://opensource.org/licenses/BSD-2-Clause diff --git a/spark/v3.3/spark-runtime/LICENSE b/spark/v3.3/spark-runtime/LICENSE index 1d3e877720d7..4ab20a3a49c9 100644 --- a/spark/v3.3/spark-runtime/LICENSE +++ b/spark/v3.3/spark-runtime/LICENSE @@ -637,3 +637,11 @@ This product includes code from Apache HttpComponents Client. Copyright: 1999-2022 The Apache Software Foundation. Home page: https://hc.apache.org/ License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains failsafe. + +Copyright: Jonathan Halterman and friends +Home page: https://failsafe.dev/ +License: https://www.apache.org/licenses/LICENSE-2.0.html \ No newline at end of file diff --git a/spark/v3.4/spark-runtime/LICENSE b/spark/v3.4/spark-runtime/LICENSE index 1d3e877720d7..4ab20a3a49c9 100644 --- a/spark/v3.4/spark-runtime/LICENSE +++ b/spark/v3.4/spark-runtime/LICENSE @@ -637,3 +637,11 @@ This product includes code from Apache HttpComponents Client. Copyright: 1999-2022 The Apache Software Foundation. Home page: https://hc.apache.org/ License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains failsafe. + +Copyright: Jonathan Halterman and friends +Home page: https://failsafe.dev/ +License: https://www.apache.org/licenses/LICENSE-2.0.html \ No newline at end of file diff --git a/spark/v3.5/spark-runtime/LICENSE b/spark/v3.5/spark-runtime/LICENSE index 1d3e877720d7..4ab20a3a49c9 100644 --- a/spark/v3.5/spark-runtime/LICENSE +++ b/spark/v3.5/spark-runtime/LICENSE @@ -637,3 +637,11 @@ This product includes code from Apache HttpComponents Client. Copyright: 1999-2022 The Apache Software Foundation. Home page: https://hc.apache.org/ License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains failsafe. + +Copyright: Jonathan Halterman and friends +Home page: https://failsafe.dev/ +License: https://www.apache.org/licenses/LICENSE-2.0.html \ No newline at end of file From 3535240c38e17fdef4a1d0a5633422186848f88c Mon Sep 17 00:00:00 2001 From: Mingliang Liu Date: Thu, 19 Dec 2024 10:58:22 -0800 Subject: [PATCH 276/313] Docs: Change to Flink directory for instructions (#11031) --- docs/docs/flink.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/docs/flink.md b/docs/docs/flink.md index 0dd427a8ff51..e45a760c37fb 100644 --- a/docs/docs/flink.md +++ b/docs/docs/flink.md @@ -66,6 +66,7 @@ HADOOP_HOME=`pwd`/hadoop-${HADOOP_VERSION} export HADOOP_CLASSPATH=`$HADOOP_HOME/bin/hadoop classpath` # Start the flink standalone cluster +cd flink-${FLINK_VERSION}/ ./bin/start-cluster.sh ``` From 70336679e12d243e3ff0a1962bb21337166e77ff Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 19 Dec 2024 11:36:49 -0800 Subject: [PATCH 277/313] Spark 3.5: Support default values in vectorized reads (#11815) --- .../vectorized/VectorizedReaderBuilder.java | 33 +++++++- .../VectorizedSparkParquetReaders.java | 9 +- .../iceberg/spark/data/AvroDataTest.java | 35 +++++--- .../iceberg/spark/data/TestHelpers.java | 30 +++++-- .../TestParquetVectorizedReads.java | 82 +++++++++---------- 5 files changed, 125 insertions(+), 64 deletions(-) diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java index 3915ff1f1a32..398f42eb1ce7 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.IntStream; import org.apache.arrow.memory.BufferAllocator; @@ -47,6 +48,7 @@ public class VectorizedReaderBuilder extends TypeWithSchemaVisitor idToConstant; private final boolean setArrowValidityVector; private final Function>, VectorizedReader> readerFactory; + private final BiFunction convert; public VectorizedReaderBuilder( Schema expectedSchema, @@ -54,6 +56,22 @@ public VectorizedReaderBuilder( boolean setArrowValidityVector, Map idToConstant, Function>, VectorizedReader> readerFactory) { + this( + expectedSchema, + parquetSchema, + setArrowValidityVector, + idToConstant, + readerFactory, + (type, value) -> value); + } + + protected VectorizedReaderBuilder( + Schema expectedSchema, + MessageType parquetSchema, + boolean setArrowValidityVector, + Map idToConstant, + Function>, VectorizedReader> readerFactory, + BiFunction convert) { this.parquetSchema = parquetSchema; this.icebergSchema = expectedSchema; this.rootAllocator = @@ -62,6 +80,7 @@ public VectorizedReaderBuilder( this.setArrowValidityVector = setArrowValidityVector; this.idToConstant = idToConstant; this.readerFactory = readerFactory; + this.convert = convert; } @Override @@ -85,7 +104,7 @@ public VectorizedReader message( int id = field.fieldId(); VectorizedReader reader = readersById.get(id); if (idToConstant.containsKey(id)) { - reorderedFields.add(new ConstantVectorReader<>(field, idToConstant.get(id))); + reorderedFields.add(constantReader(field, idToConstant.get(id))); } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { if (setArrowValidityVector) { reorderedFields.add(VectorizedArrowReader.positionsWithSetArrowValidityVector()); @@ -96,13 +115,23 @@ public VectorizedReader message( reorderedFields.add(new DeletedVectorReader()); } else if (reader != null) { reorderedFields.add(reader); - } else { + } else if (field.initialDefault() != null) { + reorderedFields.add( + constantReader(field, convert.apply(field.type(), field.initialDefault()))); + } else if (field.isOptional()) { reorderedFields.add(VectorizedArrowReader.nulls()); + } else { + throw new IllegalArgumentException( + String.format("Missing required field: %s", field.name())); } } return vectorizedReader(reorderedFields); } + private ConstantVectorReader constantReader(Types.NestedField field, T constant) { + return new ConstantVectorReader<>(field, constant); + } + protected VectorizedReader vectorizedReader(List> reorderedFields) { return readerFactory.apply(reorderedFields); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index e47152c79398..636ad3be7dcc 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -27,6 +27,7 @@ import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.TypeWithSchemaVisitor; import org.apache.iceberg.parquet.VectorizedReader; +import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; import org.apache.spark.sql.catalyst.InternalRow; import org.slf4j.Logger; @@ -112,7 +113,13 @@ private static class ReaderBuilder extends VectorizedReaderBuilder { Map idToConstant, Function>, VectorizedReader> readerFactory, DeleteFilter deleteFilter) { - super(expectedSchema, parquetSchema, setArrowValidityVector, idToConstant, readerFactory); + super( + expectedSchema, + parquetSchema, + setArrowValidityVector, + idToConstant, + readerFactory, + SparkUtil::internalToSpark); this.deleteFilter = deleteFilter; } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java index 4f7eab30a47d..d6e8ae773b4b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java @@ -63,6 +63,10 @@ protected boolean supportsDefaultValues() { return false; } + protected boolean supportsNestedTypes() { + return true; + } + protected static final StructType SUPPORTED_PRIMITIVES = StructType.of( required(100, "id", LongType.get()), @@ -74,6 +78,7 @@ protected boolean supportsDefaultValues() { required(106, "d", Types.DoubleType.get()), optional(107, "date", Types.DateType.get()), required(108, "ts", Types.TimestampType.withZone()), + required(109, "ts_without_zone", Types.TimestampType.withoutZone()), required(110, "s", Types.StringType.get()), required(111, "uuid", Types.UUIDType.get()), required(112, "fixed", Types.FixedType.ofLength(7)), @@ -109,12 +114,16 @@ public void testStructWithOptionalFields() throws IOException { @Test public void testNestedStruct() throws IOException { + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); + writeAndValidate( TypeUtil.assignIncreasingFreshIds(new Schema(required(1, "struct", SUPPORTED_PRIMITIVES)))); } @Test public void testArray() throws IOException { + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); + Schema schema = new Schema( required(0, "id", LongType.get()), @@ -125,6 +134,8 @@ public void testArray() throws IOException { @Test public void testArrayOfStructs() throws IOException { + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); + Schema schema = TypeUtil.assignIncreasingFreshIds( new Schema( @@ -136,6 +147,8 @@ public void testArrayOfStructs() throws IOException { @Test public void testMap() throws IOException { + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); + Schema schema = new Schema( required(0, "id", LongType.get()), @@ -149,6 +162,8 @@ public void testMap() throws IOException { @Test public void testNumericMapKey() throws IOException { + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); + Schema schema = new Schema( required(0, "id", LongType.get()), @@ -160,6 +175,8 @@ public void testNumericMapKey() throws IOException { @Test public void testComplexMapKey() throws IOException { + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); + Schema schema = new Schema( required(0, "id", LongType.get()), @@ -179,6 +196,8 @@ public void testComplexMapKey() throws IOException { @Test public void testMapOfStructs() throws IOException { + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); + Schema schema = TypeUtil.assignIncreasingFreshIds( new Schema( @@ -193,6 +212,8 @@ public void testMapOfStructs() throws IOException { @Test public void testMixedTypes() throws IOException { + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); + StructType structType = StructType.of( required(0, "id", LongType.get()), @@ -248,17 +269,6 @@ public void testMixedTypes() throws IOException { writeAndValidate(schema); } - @Test - public void testTimestampWithoutZone() throws IOException { - Schema schema = - TypeUtil.assignIncreasingFreshIds( - new Schema( - required(0, "id", LongType.get()), - optional(1, "ts_without_zone", Types.TimestampType.withoutZone()))); - - writeAndValidate(schema); - } - @Test public void testMissingRequiredWithoutDefault() { Assumptions.assumeThat(supportsDefaultValues()).isTrue(); @@ -348,6 +358,7 @@ public void testNullDefaultValue() throws IOException { @Test public void testNestedDefaultValue() throws IOException { Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); Schema writeSchema = new Schema( @@ -391,6 +402,7 @@ public void testNestedDefaultValue() throws IOException { @Test public void testMapNestedDefaultValue() throws IOException { Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); Schema writeSchema = new Schema( @@ -443,6 +455,7 @@ public void testMapNestedDefaultValue() throws IOException { @Test public void testListNestedDefaultValue() throws IOException { Assumptions.assumeThat(supportsDefaultValues()).isTrue(); + Assumptions.assumeThat(supportsNestedTypes()).isTrue(); Schema writeSchema = new Schema( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index d3d69e4b9d86..64d0b85625a9 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -79,6 +79,8 @@ import org.apache.spark.sql.types.MapType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.TimestampNTZType; +import org.apache.spark.sql.types.TimestampType$; import org.apache.spark.sql.vectorized.ColumnarBatch; import org.apache.spark.unsafe.types.UTF8String; import scala.collection.Seq; @@ -107,13 +109,25 @@ public static void assertEqualsSafe(Types.StructType struct, Record rec, Row row public static void assertEqualsBatch( Types.StructType struct, Iterator expected, ColumnarBatch batch) { for (int rowId = 0; rowId < batch.numRows(); rowId++) { - List fields = struct.fields(); InternalRow row = batch.getRow(rowId); Record rec = expected.next(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i).type(); - Object expectedValue = rec.get(i); - Object actualValue = row.isNullAt(i) ? null : row.get(i, convert(fieldType)); + + List fields = struct.fields(); + for (int readPos = 0; readPos < fields.size(); readPos += 1) { + Types.NestedField field = fields.get(readPos); + Field writeField = rec.getSchema().getField(field.name()); + + Type fieldType = field.type(); + Object actualValue = row.isNullAt(readPos) ? null : row.get(readPos, convert(fieldType)); + + Object expectedValue; + if (writeField != null) { + int writePos = writeField.pos(); + expectedValue = rec.get(writePos); + } else { + expectedValue = field.initialDefault(); + } + assertEqualsUnsafe(fieldType, expectedValue, actualValue); } } @@ -751,6 +765,12 @@ private static void assertEquals( for (int i = 0; i < actual.numFields(); i += 1) { StructField field = struct.fields()[i]; DataType type = field.dataType(); + // ColumnarRow.get doesn't support TimestampNTZType, causing tests to fail. the representation + // is identical to TimestampType so this uses that type to validate. + if (type instanceof TimestampNTZType) { + type = TimestampType$.MODULE$; + } + assertEquals( context + "." + field.name(), type, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java index 5c4b216aff94..4f7864e9a160 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java @@ -49,7 +49,6 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; import org.apache.spark.sql.vectorized.ColumnarBatch; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; public class TestParquetVectorizedReads extends AvroDataTest { @@ -60,18 +59,42 @@ public class TestParquetVectorizedReads extends AvroDataTest { @Override protected void writeAndValidate(Schema schema) throws IOException { - writeAndValidate(schema, getNumRows(), 0L, RandomData.DEFAULT_NULL_PERCENTAGE, true); + writeAndValidate(schema, schema); + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + writeAndValidate( + writeSchema, + expectedSchema, + getNumRows(), + 29714278L, + RandomData.DEFAULT_NULL_PERCENTAGE, + true, + BATCH_SIZE, + IDENTITY); + } + + @Override + protected boolean supportsDefaultValues() { + return true; + } + + @Override + protected boolean supportsNestedTypes() { + return false; } private void writeAndValidate( Schema schema, int numRecords, long seed, float nullPercentage, boolean reuseContainers) throws IOException { writeAndValidate( - schema, numRecords, seed, nullPercentage, reuseContainers, BATCH_SIZE, IDENTITY); + schema, schema, numRecords, seed, nullPercentage, reuseContainers, BATCH_SIZE, IDENTITY); } private void writeAndValidate( - Schema schema, + Schema writeSchema, + Schema expectedSchema, int numRecords, long seed, float nullPercentage, @@ -82,22 +105,23 @@ private void writeAndValidate( // Write test data assumeThat( TypeUtil.find( - schema, + writeSchema, type -> type.isMapType() && type.asMapType().keyType() != Types.StringType.get())) .as("Parquet Avro cannot write non-string map keys") .isNull(); Iterable expected = - generateData(schema, numRecords, seed, nullPercentage, transform); + generateData(writeSchema, numRecords, seed, nullPercentage, transform); // write a test parquet file using iceberg writer File testFile = File.createTempFile("junit", null, temp.toFile()); assertThat(testFile.delete()).as("Delete should succeed").isTrue(); - try (FileAppender writer = getParquetWriter(schema, testFile)) { + try (FileAppender writer = getParquetWriter(writeSchema, testFile)) { writer.addAll(expected); } - assertRecordsMatch(schema, numRecords, expected, testFile, reuseContainers, batchSize); + + assertRecordsMatch(expectedSchema, numRecords, expected, testFile, reuseContainers, batchSize); } protected int getNumRows() { @@ -161,41 +185,6 @@ void assertRecordsMatch( } } - @Override - @Test - @Disabled - public void testArray() {} - - @Override - @Test - @Disabled - public void testArrayOfStructs() {} - - @Override - @Test - @Disabled - public void testMap() {} - - @Override - @Test - @Disabled - public void testNumericMapKey() {} - - @Override - @Test - @Disabled - public void testComplexMapKey() {} - - @Override - @Test - @Disabled - public void testMapOfStructs() {} - - @Override - @Test - @Disabled - public void testMixedTypes() {} - @Test @Override public void testNestedStruct() { @@ -246,10 +235,13 @@ public void testVectorizedReadsWithNewContainers() throws IOException { public void testVectorizedReadsWithReallocatedArrowBuffers() throws IOException { // With a batch size of 2, 256 bytes are allocated in the VarCharVector. By adding strings of // length 512, the vector will need to be reallocated for storing the batch. - writeAndValidate( + Schema schema = new Schema( Lists.newArrayList( - SUPPORTED_PRIMITIVES.field("id"), SUPPORTED_PRIMITIVES.field("data"))), + SUPPORTED_PRIMITIVES.field("id"), SUPPORTED_PRIMITIVES.field("data"))); + writeAndValidate( + schema, + schema, 10, 0L, RandomData.DEFAULT_NULL_PERCENTAGE, From ed36a9f9a3943db989645317fc563ad56e50484e Mon Sep 17 00:00:00 2001 From: Tan Qi Date: Thu, 19 Dec 2024 23:28:33 -0800 Subject: [PATCH 278/313] Spark 3.5: Remove numbers from assert description in TestRewritePositionDeleteFilesAction (#11827) --- .../TestRewritePositionDeleteFilesAction.java | 82 +++++++++---------- 1 file changed, 39 insertions(+), 43 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 21719657d187..8ecec5ac2d42 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -194,7 +194,7 @@ public void testUnpartitioned() throws Exception { .option(SizeBasedFileRewriter.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); - assertThat(newDeleteFiles).as("Expected 1 new delete file").hasSize(1); + assertThat(newDeleteFiles).as("New delete files").hasSize(1); assertLocallySorted(newDeleteFiles); assertNotContains(deleteFiles, newDeleteFiles); checkResult(result, deleteFiles, newDeleteFiles, 1); @@ -202,8 +202,8 @@ public void testUnpartitioned() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + assertEquals("Rows", expectedRecords, actualRecords); + assertEquals("Position deletes", expectedDeletes, actualDeletes); } @TestTemplate @@ -238,8 +238,8 @@ public void testRewriteAll() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + assertEquals("Rows", expectedRecords, actualRecords); + assertEquals("Position deletes", expectedDeletes, actualDeletes); } @TestTemplate @@ -284,8 +284,8 @@ public void testRewriteFilter() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + assertEquals("Rows", expectedRecords, actualRecords); + assertEquals("Position deletes", expectedDeletes, actualDeletes); withSQLConf( ImmutableMap.of(SQLConf.CASE_SENSITIVE().key(), "true"), @@ -326,7 +326,7 @@ public void testRewriteToSmallerTarget() throws Exception { .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, String.valueOf(avgSize / 2)) .execute(); List newDeleteFiles = deleteFiles(table); - assertThat(newDeleteFiles).as("Should have 8 new delete files").hasSize(8); + assertThat(newDeleteFiles).as("New delete files").hasSize(8); assertNotContains(deleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, deleteFiles, newDeleteFiles, 4); @@ -334,8 +334,8 @@ public void testRewriteToSmallerTarget() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + assertEquals("Rows", expectedRecords, actualRecords); + assertEquals("Position deletes", expectedDeletes, actualDeletes); } @TestTemplate @@ -371,7 +371,7 @@ public void testRemoveDanglingDeletes() throws Exception { .option(SizeBasedFileRewriter.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); - assertThat(newDeleteFiles).as("Should have 0 new delete files").hasSize(0); + assertThat(newDeleteFiles).as("New delete files").isEmpty(); assertNotContains(deleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, deleteFiles, newDeleteFiles, 4); @@ -379,8 +379,8 @@ public void testRemoveDanglingDeletes() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertThat(actualDeletes).as("Should be no new position deletes").hasSize(0); + assertEquals("Rows", expectedRecords, actualRecords); + assertThat(actualDeletes).as("New position deletes").isEmpty(); } @TestTemplate @@ -413,7 +413,7 @@ public void testSomePartitionsDanglingDeletes() throws Exception { .option(SizeBasedFileRewriter.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); - assertThat(newDeleteFiles).as("Should have 2 new delete files").hasSize(2); + assertThat(newDeleteFiles).as("New delete files").hasSize(2); assertNotContains(deleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, deleteFiles, newDeleteFiles, 4); @@ -432,8 +432,8 @@ public void testSomePartitionsDanglingDeletes() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + assertEquals("Rows", expectedRecords, actualRecords); + assertEquals("Position deletes", expectedDeletes, actualDeletes); } @TestTemplate @@ -469,7 +469,7 @@ public void testRewriteFilterRemoveDangling() throws Exception { .execute(); List newDeleteFiles = except(deleteFiles(table), deleteFiles); - assertThat(newDeleteFiles).as("New delete files").hasSize(0); + assertThat(newDeleteFiles).as("New delete files").isEmpty(); List expectedRewrittenFiles = filterFiles(table, deleteFiles, ImmutableList.of(0), ImmutableList.of(1)); @@ -480,8 +480,8 @@ public void testRewriteFilterRemoveDangling() throws Exception { // Only non-compacted deletes remain List expectedDeletesFiltered = filterDeletes(expectedDeletes, ImmutableList.of(2), ImmutableList.of(3)); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertEquals("Position deletes must match", expectedDeletesFiltered, allDeletes); + assertEquals("Rows", expectedRecords, actualRecords); + assertEquals("Position deletes", expectedDeletesFiltered, allDeletes); } @TestTemplate @@ -524,7 +524,7 @@ public void testPartitionEvolutionAdd() throws Exception { Stream.concat(unpartitionedDeleteFiles.stream(), partitionedDeleteFiles.stream()) .collect(Collectors.toList()); List newDeleteFiles = deleteFiles(table); - assertThat(newDeleteFiles).as("Should have 3 new delete files").hasSize(3); + assertThat(newDeleteFiles).as("New delete files").hasSize(3); assertNotContains(rewrittenDeleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, rewrittenDeleteFiles, newDeleteFiles, 3); @@ -532,8 +532,8 @@ public void testPartitionEvolutionAdd() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + assertEquals("Rows", expectedRecords, actualRecords); + assertEquals("Position deletes", expectedDeletes, actualDeletes); } @TestTemplate @@ -571,7 +571,7 @@ public void testPartitionEvolutionRemove() throws Exception { .option(SizeBasedFileRewriter.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); - assertThat(newDeleteFiles).as("Should have 3 new delete files").hasSize(3); + assertThat(newDeleteFiles).as("New delete files").hasSize(3); assertNotContains(expectedRewritten, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, expectedRewritten, newDeleteFiles, 3); @@ -579,8 +579,8 @@ public void testPartitionEvolutionRemove() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + assertEquals("Rows", expectedRecords, actualRecords); + assertEquals("Position deletes", expectedDeletes, actualDeletes); } @TestTemplate @@ -622,14 +622,14 @@ public void testSchemaEvolution() throws Exception { Stream.concat(deleteFiles.stream(), newSchemaDeleteFiles.stream()) .collect(Collectors.toList()); List newDeleteFiles = deleteFiles(table); - assertThat(newDeleteFiles).as("Should have 2 new delete files").hasSize(4); + assertThat(newDeleteFiles).as("New delete files").hasSize(4); assertNotContains(rewrittenDeleteFiles, newDeleteFiles); assertLocallySorted(newDeleteFiles); checkResult(result, rewrittenDeleteFiles, newDeleteFiles, 4); checkSequenceNumbers(table, rewrittenDeleteFiles, newDeleteFiles); List actualRecords = records(table); - assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Rows", expectedRecords, actualRecords); } @TestTemplate @@ -724,8 +724,8 @@ public void testRewriteManyColumns() throws Exception { List actualRecords = records(table); List actualDeletes = deleteRecords(table); - assertEquals("Rows must match", expectedRecords, actualRecords); - assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + assertEquals("Rows", expectedRecords, actualRecords); + assertEquals("Position deletes", expectedDeletes, actualDeletes); } private Table createTablePartitioned(int partitions, int files, int numRecords) { @@ -948,7 +948,7 @@ private void assertNotContains(List original, List rewri Set rewrittenPaths = rewritten.stream().map(ContentFile::location).collect(Collectors.toSet()); rewrittenPaths.retainAll(originalPaths); - assertThat(rewrittenPaths).hasSize(0); + assertThat(rewrittenPaths).isEmpty(); } private void assertLocallySorted(List deleteFiles) { @@ -1054,48 +1054,44 @@ private void checkResult( List newDeletes, int expectedGroups) { assertThat(rewrittenDeletes.size()) - .as("Expected rewritten delete file count does not match") + .as("Rewritten delete file count") .isEqualTo(result.rewrittenDeleteFilesCount()); assertThat(newDeletes.size()) - .as("Expected new delete file count does not match") + .as("New delete file count") .isEqualTo(result.addedDeleteFilesCount()); assertThat(size(rewrittenDeletes)) - .as("Expected rewritten delete byte count does not match") + .as("Rewritten delete byte count") .isEqualTo(result.rewrittenBytesCount()); - assertThat(size(newDeletes)) - .as("Expected new delete byte count does not match") - .isEqualTo(result.addedBytesCount()); + assertThat(size(newDeletes)).as("New delete byte count").isEqualTo(result.addedBytesCount()); - assertThat(expectedGroups) - .as("Expected rewrite group count does not match") - .isEqualTo(result.rewriteResults().size()); + assertThat(expectedGroups).as("Rewrite group count").isEqualTo(result.rewriteResults().size()); assertThat(rewrittenDeletes.size()) - .as("Expected rewritten delete file count in all groups to match") + .as("Rewritten delete file count in all groups") .isEqualTo( result.rewriteResults().stream() .mapToInt(FileGroupRewriteResult::rewrittenDeleteFilesCount) .sum()); assertThat(newDeletes.size()) - .as("Expected added delete file count in all groups to match") + .as("Added delete file count in all groups") .isEqualTo( result.rewriteResults().stream() .mapToInt(FileGroupRewriteResult::addedDeleteFilesCount) .sum()); assertThat(size(rewrittenDeletes)) - .as("Expected rewritten delete bytes in all groups to match") + .as("Rewritten delete bytes in all groups") .isEqualTo( result.rewriteResults().stream() .mapToLong(FileGroupRewriteResult::rewrittenBytesCount) .sum()); assertThat(size(newDeletes)) - .as("Expected added delete bytes in all groups to match") + .as("Added delete bytes in all groups") .isEqualTo( result.rewriteResults().stream() .mapToLong(FileGroupRewriteResult::addedBytesCount) From cdf748e8e5537f13d861aa4c617a51f3e11dc97c Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Fri, 20 Dec 2024 18:51:51 +0100 Subject: [PATCH 279/313] Auth Manager API part 2: AuthManager (#11809) * Auth Manager API part 2: AuthManager, AuthSession * [review] close() * [review] nits * HTTPAuthSession close() * mainSession -> catalogSession * AuthManager constructor --- .../org/apache/iceberg/rest/HTTPHeaders.java | 10 ++ .../apache/iceberg/rest/auth/AuthManager.java | 106 ++++++++++++++++++ .../iceberg/rest/auth/AuthManagers.java | 75 +++++++++++++ .../iceberg/rest/auth/AuthProperties.java | 37 ++++++ .../apache/iceberg/rest/auth/AuthSession.java | 57 ++++++++++ .../iceberg/rest/auth/BasicAuthManager.java | 53 +++++++++ .../iceberg/rest/auth/DefaultAuthSession.java | 57 ++++++++++ .../iceberg/rest/auth/NoopAuthManager.java | 40 +++++++ .../apache/iceberg/rest/TestHTTPHeaders.java | 12 ++ .../iceberg/rest/auth/TestAuthManagers.java | 88 +++++++++++++++ .../rest/auth/TestBasicAuthManager.java | 62 ++++++++++ .../rest/auth/TestDefaultAuthSession.java | 72 ++++++++++++ 12 files changed, 669 insertions(+) create mode 100644 core/src/main/java/org/apache/iceberg/rest/auth/AuthManager.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/auth/AuthManagers.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/auth/AuthProperties.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/auth/AuthSession.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/auth/BasicAuthManager.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/auth/DefaultAuthSession.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/auth/NoopAuthManager.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/auth/TestAuthManagers.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/auth/TestBasicAuthManager.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/auth/TestDefaultAuthSession.java diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPHeaders.java b/core/src/main/java/org/apache/iceberg/rest/HTTPHeaders.java index 35710bd9a9b7..23263899b95f 100644 --- a/core/src/main/java/org/apache/iceberg/rest/HTTPHeaders.java +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPHeaders.java @@ -19,6 +19,7 @@ package org.apache.iceberg.rest; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -85,6 +86,15 @@ static HTTPHeaders of(HTTPHeader... headers) { return ImmutableHTTPHeaders.builder().addEntries(headers).build(); } + static HTTPHeaders of(Map headers) { + return ImmutableHTTPHeaders.builder() + .entries( + headers.entrySet().stream() + .map(e -> HTTPHeader.of(e.getKey(), e.getValue())) + .collect(Collectors.toList())) + .build(); + } + /** Represents an HTTP header as a name-value pair. */ @Value.Style(redactedMask = "****", depluralize = true) @Value.Immutable diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/AuthManager.java b/core/src/main/java/org/apache/iceberg/rest/auth/AuthManager.java new file mode 100644 index 000000000000..8f6f16f925e3 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/auth/AuthManager.java @@ -0,0 +1,106 @@ +/* + * 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.rest.auth; + +import java.util.Map; +import org.apache.iceberg.catalog.SessionCatalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.rest.RESTClient; + +/** + * Manager for authentication sessions. This interface is used to create sessions for the catalog, + * the tables/views, and any other context that requires authentication. + * + *

    Managers are usually stateful and may require initialization and cleanup. The manager is + * created by the catalog and is closed when the catalog is closed. + */ +public interface AuthManager extends AutoCloseable { + + /** + * Returns a temporary session to use for contacting the configuration endpoint only. Note that + * the returned session will be closed after the configuration endpoint is contacted, and should + * not be cached. + * + *

    The provided REST client is a short-lived client; it should only be used to fetch initial + * credentials, if required, and must be discarded after that. + * + *

    This method cannot return null. By default, it returns the catalog session. + */ + default AuthSession initSession(RESTClient initClient, Map properties) { + return catalogSession(initClient, properties); + } + + /** + * Returns a long-lived session whose lifetime is tied to the owning catalog. This session serves + * as the parent session for all other sessions (contextual and table-specific). It is closed when + * the owning catalog is closed. + * + *

    The provided REST client is a long-lived, shared client; if required, implementors may store + * it and reuse it for all subsequent requests to the authorization server, e.g. for renewing or + * refreshing credentials. It is not necessary to close it when {@link #close()} is called. + * + *

    This method cannot return null. + * + *

    It is not required to cache the returned session internally, as the catalog will keep it + * alive for the lifetime of the catalog. + */ + AuthSession catalogSession(RESTClient sharedClient, Map properties); + + /** + * Returns a session for a specific context. + * + *

    If the context requires a specific {@link AuthSession}, this method should return a new + * {@link AuthSession} instance, otherwise it should return the parent session. + * + *

    This method cannot return null. By default, it returns the parent session. + * + *

    Implementors should cache contextual sessions internally, as the catalog will not cache + * them. Also, the owning catalog never closes contextual sessions; implementations should manage + * their lifecycle themselves and close them when they are no longer needed. + */ + default AuthSession contextualSession(SessionCatalog.SessionContext context, AuthSession parent) { + return parent; + } + + /** + * Returns a new session targeting a specific table or view. The properties are the ones returned + * by the table/view endpoint. + * + *

    If the table or view requires a specific {@link AuthSession}, this method should return a + * new {@link AuthSession} instance, otherwise it should return the parent session. + * + *

    This method cannot return null. By default, it returns the parent session. + * + *

    Implementors should cache table sessions internally, as the catalog will not cache them. + * Also, the owning catalog never closes table sessions; implementations should manage their + * lifecycle themselves and close them when they are no longer needed. + */ + default AuthSession tableSession( + TableIdentifier table, Map properties, AuthSession parent) { + return parent; + } + + /** + * Closes the manager and releases any resources. + * + *

    This method is called when the owning catalog is closed. + */ + @Override + void close(); +} diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/AuthManagers.java b/core/src/main/java/org/apache/iceberg/rest/auth/AuthManagers.java new file mode 100644 index 000000000000..42c2b1eeba83 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/auth/AuthManagers.java @@ -0,0 +1,75 @@ +/* + * 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.rest.auth; + +import java.util.Locale; +import java.util.Map; +import org.apache.iceberg.common.DynConstructors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AuthManagers { + + private static final Logger LOG = LoggerFactory.getLogger(AuthManagers.class); + + private AuthManagers() {} + + public static AuthManager loadAuthManager(String name, Map properties) { + String authType = + properties.getOrDefault(AuthProperties.AUTH_TYPE, AuthProperties.AUTH_TYPE_NONE); + + String impl; + switch (authType.toLowerCase(Locale.ROOT)) { + case AuthProperties.AUTH_TYPE_NONE: + impl = AuthProperties.AUTH_MANAGER_IMPL_NONE; + break; + case AuthProperties.AUTH_TYPE_BASIC: + impl = AuthProperties.AUTH_MANAGER_IMPL_BASIC; + break; + default: + impl = authType; + } + + LOG.info("Loading AuthManager implementation: {}", impl); + DynConstructors.Ctor ctor; + try { + ctor = + DynConstructors.builder(AuthManager.class) + .loader(AuthManagers.class.getClassLoader()) + .impl(impl, String.class) // with name + .buildChecked(); + } catch (NoSuchMethodException e) { + throw new IllegalArgumentException( + String.format( + "Cannot initialize AuthManager implementation %s: %s", impl, e.getMessage()), + e); + } + + AuthManager authManager; + try { + authManager = ctor.newInstance(name); + } catch (ClassCastException e) { + throw new IllegalArgumentException( + String.format("Cannot initialize AuthManager, %s does not implement AuthManager", impl), + e); + } + + return authManager; + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/AuthProperties.java b/core/src/main/java/org/apache/iceberg/rest/auth/AuthProperties.java new file mode 100644 index 000000000000..bf94311d5578 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/auth/AuthProperties.java @@ -0,0 +1,37 @@ +/* + * 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.rest.auth; + +public final class AuthProperties { + + private AuthProperties() {} + + public static final String AUTH_TYPE = "rest.auth.type"; + + public static final String AUTH_TYPE_NONE = "none"; + public static final String AUTH_TYPE_BASIC = "basic"; + + public static final String AUTH_MANAGER_IMPL_NONE = + "org.apache.iceberg.rest.auth.NoopAuthManager"; + public static final String AUTH_MANAGER_IMPL_BASIC = + "org.apache.iceberg.rest.auth.BasicAuthManager"; + + public static final String BASIC_USERNAME = "rest.auth.basic.username"; + public static final String BASIC_PASSWORD = "rest.auth.basic.password"; +} diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/AuthSession.java b/core/src/main/java/org/apache/iceberg/rest/auth/AuthSession.java new file mode 100644 index 000000000000..eed7caf84572 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/auth/AuthSession.java @@ -0,0 +1,57 @@ +/* + * 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.rest.auth; + +import org.apache.iceberg.rest.HTTPRequest; + +/** + * An authentication session that can be used to authenticate outgoing HTTP requests. + * + *

    Authentication sessions are usually immutable, but may hold resources that need to be released + * when the session is no longer needed. Implementations should override {@link #close()} to release + * any resources. + */ +public interface AuthSession extends AutoCloseable { + + /** An empty session that does nothing. */ + AuthSession EMPTY = + new AuthSession() { + @Override + public HTTPRequest authenticate(HTTPRequest request) { + return request; + } + + @Override + public void close() {} + }; + + /** + * Authenticates the given request and returns a new request with the necessary authentication. + */ + HTTPRequest authenticate(HTTPRequest request); + + /** + * Closes the session and releases any resources. This method is called when the session is no + * longer needed. Note that since sessions may be cached, this method may not be called + * immediately after the session is no longer needed, but rather when the session is evicted from + * the cache, or the cache itself is closed. + */ + @Override + void close(); +} diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/BasicAuthManager.java b/core/src/main/java/org/apache/iceberg/rest/auth/BasicAuthManager.java new file mode 100644 index 000000000000..d0d56d3d3794 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/auth/BasicAuthManager.java @@ -0,0 +1,53 @@ +/* + * 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.rest.auth; + +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.HTTPHeaders; +import org.apache.iceberg.rest.RESTClient; + +/** An auth manager that adds static BASIC authentication data to outgoing HTTP requests. */ +public final class BasicAuthManager implements AuthManager { + + public BasicAuthManager(String ignored) { + // no-op + } + + @Override + public AuthSession catalogSession(RESTClient sharedClient, Map properties) { + Preconditions.checkArgument( + properties.containsKey(AuthProperties.BASIC_USERNAME), + "Invalid username: missing required property %s", + AuthProperties.BASIC_USERNAME); + Preconditions.checkArgument( + properties.containsKey(AuthProperties.BASIC_PASSWORD), + "Invalid password: missing required property %s", + AuthProperties.BASIC_PASSWORD); + String username = properties.get(AuthProperties.BASIC_USERNAME); + String password = properties.get(AuthProperties.BASIC_PASSWORD); + String credentials = username + ":" + password; + return DefaultAuthSession.of(HTTPHeaders.of(OAuth2Util.basicAuthHeaders(credentials))); + } + + @Override + public void close() { + // no resources to close + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/DefaultAuthSession.java b/core/src/main/java/org/apache/iceberg/rest/auth/DefaultAuthSession.java new file mode 100644 index 000000000000..002f47459dd7 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/auth/DefaultAuthSession.java @@ -0,0 +1,57 @@ +/* + * 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.rest.auth; + +import org.apache.iceberg.rest.HTTPHeaders; +import org.apache.iceberg.rest.HTTPRequest; +import org.apache.iceberg.rest.ImmutableHTTPRequest; +import org.immutables.value.Value; + +/** + * Default implementation of {@link AuthSession}. It authenticates requests by setting the provided + * headers on the request. + * + *

    Most {@link AuthManager} implementations should make use of this class, unless they need to + * retain state when creating sessions, or if they need to modify the request in a different way. + */ +@Value.Style(redactedMask = "****") +@Value.Immutable +@SuppressWarnings({"ImmutablesStyle", "SafeLoggingPropagation"}) +public interface DefaultAuthSession extends AuthSession { + + /** Headers containing authentication data to set on the request. */ + HTTPHeaders headers(); + + @Override + default HTTPRequest authenticate(HTTPRequest request) { + HTTPHeaders headers = request.headers().putIfAbsent(headers()); + return headers.equals(request.headers()) + ? request + : ImmutableHTTPRequest.builder().from(request).headers(headers).build(); + } + + @Override + default void close() { + // no resources to close + } + + static DefaultAuthSession of(HTTPHeaders headers) { + return ImmutableDefaultAuthSession.builder().headers(headers).build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/NoopAuthManager.java b/core/src/main/java/org/apache/iceberg/rest/auth/NoopAuthManager.java new file mode 100644 index 000000000000..d706d78ef3ae --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/auth/NoopAuthManager.java @@ -0,0 +1,40 @@ +/* + * 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.rest.auth; + +import java.util.Map; +import org.apache.iceberg.rest.RESTClient; + +/** An auth manager that does not add any authentication data to outgoing HTTP requests. */ +public class NoopAuthManager implements AuthManager { + + public NoopAuthManager(String ignored) { + // no-op + } + + @Override + public AuthSession catalogSession(RESTClient sharedClient, Map properties) { + return AuthSession.EMPTY; + } + + @Override + public void close() { + // no resources to close + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/TestHTTPHeaders.java b/core/src/test/java/org/apache/iceberg/rest/TestHTTPHeaders.java index 9380073f7643..a8531e6ff510 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestHTTPHeaders.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestHTTPHeaders.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.rest.HTTPHeaders.HTTPHeader; import org.junit.jupiter.api.Test; @@ -119,6 +120,17 @@ void putIfAbsentHTTPHeaders() { .hasMessage("headers"); } + @Test + void ofMap() { + HTTPHeaders actual = + HTTPHeaders.of( + ImmutableMap.of( + "header1", "value1a", + "HEADER1", "value1b", + "header2", "value2")); + assertThat(actual).isEqualTo(headers); + } + @Test void invalidHeader() { // invalid input (null name or value) diff --git a/core/src/test/java/org/apache/iceberg/rest/auth/TestAuthManagers.java b/core/src/test/java/org/apache/iceberg/rest/auth/TestAuthManagers.java new file mode 100644 index 000000000000..21bd8c1b2963 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/auth/TestAuthManagers.java @@ -0,0 +1,88 @@ +/* + * 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.rest.auth; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.Map; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestAuthManagers { + + private final PrintStream standardErr = System.err; + private final ByteArrayOutputStream streamCaptor = new ByteArrayOutputStream(); + + @BeforeEach + public void before() { + System.setErr(new PrintStream(streamCaptor)); + } + + @AfterEach + public void after() { + System.setErr(standardErr); + } + + @Test + void noop() { + try (AuthManager manager = AuthManagers.loadAuthManager("test", Map.of())) { + assertThat(manager).isInstanceOf(NoopAuthManager.class); + } + assertThat(streamCaptor.toString()) + .contains( + "Loading AuthManager implementation: org.apache.iceberg.rest.auth.NoopAuthManager"); + } + + @Test + void noopExplicit() { + try (AuthManager manager = + AuthManagers.loadAuthManager( + "test", Map.of(AuthProperties.AUTH_TYPE, AuthProperties.AUTH_TYPE_NONE))) { + assertThat(manager).isInstanceOf(NoopAuthManager.class); + } + assertThat(streamCaptor.toString()) + .contains( + "Loading AuthManager implementation: org.apache.iceberg.rest.auth.NoopAuthManager"); + } + + @Test + void basicExplicit() { + try (AuthManager manager = + AuthManagers.loadAuthManager( + "test", Map.of(AuthProperties.AUTH_TYPE, AuthProperties.AUTH_TYPE_BASIC))) { + assertThat(manager).isInstanceOf(BasicAuthManager.class); + } + assertThat(streamCaptor.toString()) + .contains( + "Loading AuthManager implementation: org.apache.iceberg.rest.auth.BasicAuthManager"); + } + + @Test + @SuppressWarnings("resource") + void nonExistentAuthManager() { + assertThatThrownBy( + () -> AuthManagers.loadAuthManager("test", Map.of(AuthProperties.AUTH_TYPE, "unknown"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot initialize AuthManager implementation unknown"); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/auth/TestBasicAuthManager.java b/core/src/test/java/org/apache/iceberg/rest/auth/TestBasicAuthManager.java new file mode 100644 index 000000000000..c34654cdeff5 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/auth/TestBasicAuthManager.java @@ -0,0 +1,62 @@ +/* + * 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.rest.auth; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import org.apache.iceberg.rest.HTTPHeaders; +import org.junit.jupiter.api.Test; + +class TestBasicAuthManager { + + @Test + void missingUsername() { + try (AuthManager authManager = new BasicAuthManager("test")) { + assertThatThrownBy(() -> authManager.catalogSession(null, Map.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Invalid username: missing required property %s", AuthProperties.BASIC_USERNAME); + } + } + + @Test + void missingPassword() { + try (AuthManager authManager = new BasicAuthManager("test")) { + Map properties = Map.of(AuthProperties.BASIC_USERNAME, "alice"); + assertThatThrownBy(() -> authManager.catalogSession(null, properties)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Invalid password: missing required property %s", AuthProperties.BASIC_PASSWORD); + } + } + + @Test + void success() { + Map properties = + Map.of(AuthProperties.BASIC_USERNAME, "alice", AuthProperties.BASIC_PASSWORD, "secret"); + try (AuthManager authManager = new BasicAuthManager("test"); + AuthSession session = authManager.catalogSession(null, properties)) { + assertThat(session) + .isEqualTo( + DefaultAuthSession.of(HTTPHeaders.of(OAuth2Util.basicAuthHeaders("alice:secret")))); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/auth/TestDefaultAuthSession.java b/core/src/test/java/org/apache/iceberg/rest/auth/TestDefaultAuthSession.java new file mode 100644 index 000000000000..f6fee42e0d52 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/auth/TestDefaultAuthSession.java @@ -0,0 +1,72 @@ +/* + * 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.rest.auth; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.net.URI; +import org.apache.iceberg.rest.HTTPHeaders; +import org.apache.iceberg.rest.HTTPHeaders.HTTPHeader; +import org.apache.iceberg.rest.HTTPRequest; +import org.apache.iceberg.rest.HTTPRequest.HTTPMethod; +import org.apache.iceberg.rest.ImmutableHTTPRequest; +import org.junit.jupiter.api.Test; + +class TestDefaultAuthSession { + + @Test + void authenticate() { + try (DefaultAuthSession session = + DefaultAuthSession.of(HTTPHeaders.of(HTTPHeader.of("Authorization", "s3cr3t")))) { + + HTTPRequest original = + ImmutableHTTPRequest.builder() + .method(HTTPMethod.GET) + .baseUri(URI.create("https://localhost")) + .path("path") + .build(); + + HTTPRequest authenticated = session.authenticate(original); + + assertThat(authenticated.headers().entries()) + .singleElement() + .extracting(HTTPHeader::name, HTTPHeader::value) + .containsExactly("Authorization", "s3cr3t"); + } + } + + @Test + void authenticateWithConflictingHeader() { + try (DefaultAuthSession session = + DefaultAuthSession.of(HTTPHeaders.of(HTTPHeader.of("Authorization", "s3cr3t")))) { + + HTTPRequest original = + ImmutableHTTPRequest.builder() + .method(HTTPMethod.GET) + .baseUri(URI.create("https://localhost")) + .path("path") + .headers(HTTPHeaders.of(HTTPHeader.of("Authorization", "other"))) + .build(); + + HTTPRequest authenticated = session.authenticate(original); + + assertThat(authenticated).isSameAs(original); + } + } +} From dea2fd1d9debfd23aeda9403ed3eb81c6aebf30f Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Fri, 20 Dec 2024 13:58:43 -0800 Subject: [PATCH 280/313] Core: Add Variant implementation to read serialized objects (#11415) --- .../apache/iceberg/io/CloseableIterable.java | 17 + .../org/apache/iceberg/util/RandomUtil.java | 5 +- .../org/apache/iceberg/util/SortedMerge.java | 13 + .../iceberg/variants/PrimitiveWrapper.java | 206 ++++++++ .../iceberg/variants/SerializedArray.java | 88 ++++ .../iceberg/variants/SerializedMetadata.java | 113 +++++ .../iceberg/variants/SerializedObject.java | 229 +++++++++ .../iceberg/variants/SerializedPrimitive.java | 130 +++++ .../variants/SerializedShortString.java | 69 +++ .../iceberg/variants/ShreddedObject.java | 211 ++++++++ .../org/apache/iceberg/variants/Variant.java | 28 ++ .../apache/iceberg/variants/VariantArray.java | 35 ++ .../iceberg/variants/VariantMetadata.java | 34 ++ .../iceberg/variants/VariantObject.java | 35 ++ .../iceberg/variants/VariantPrimitive.java | 29 ++ .../apache/iceberg/variants/VariantUtil.java | 195 ++++++++ .../apache/iceberg/variants/VariantValue.java | 63 +++ .../org/apache/iceberg/variants/Variants.java | 276 +++++++++++ .../variants/TestPrimitiveWrapper.java | 82 +++ .../iceberg/variants/TestSerializedArray.java | 240 +++++++++ .../variants/TestSerializedMetadata.java | 219 +++++++++ .../variants/TestSerializedObject.java | 326 ++++++++++++ .../variants/TestSerializedPrimitives.java | 465 ++++++++++++++++++ .../iceberg/variants/TestShreddedObject.java | 448 +++++++++++++++++ .../iceberg/variants/TestVariantUtil.java | 44 ++ .../iceberg/variants/VariantTestUtil.java | 214 ++++++++ 26 files changed, 3813 insertions(+), 1 deletion(-) create mode 100644 core/src/main/java/org/apache/iceberg/variants/PrimitiveWrapper.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/SerializedArray.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/SerializedMetadata.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/SerializedObject.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/SerializedPrimitive.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/SerializedShortString.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/Variant.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/VariantArray.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/VariantMetadata.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/VariantObject.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/VariantPrimitive.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/VariantUtil.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/VariantValue.java create mode 100644 core/src/main/java/org/apache/iceberg/variants/Variants.java create mode 100644 core/src/test/java/org/apache/iceberg/variants/TestPrimitiveWrapper.java create mode 100644 core/src/test/java/org/apache/iceberg/variants/TestSerializedArray.java create mode 100644 core/src/test/java/org/apache/iceberg/variants/TestSerializedMetadata.java create mode 100644 core/src/test/java/org/apache/iceberg/variants/TestSerializedObject.java create mode 100644 core/src/test/java/org/apache/iceberg/variants/TestSerializedPrimitives.java create mode 100644 core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java create mode 100644 core/src/test/java/org/apache/iceberg/variants/TestVariantUtil.java create mode 100644 core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java diff --git a/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java b/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java index 06323612a178..34c561bc373d 100644 --- a/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java +++ b/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java @@ -32,6 +32,23 @@ public interface CloseableIterable extends Iterable, Closeable { + /** + * Adapts an Iterable to CloseableIterable using a no-op close if it is not Closeable. + * + * @param iterable an Iterable + * @return a CloseableIterable that closes Iterable if it is Closeable + */ + static CloseableIterable of(Iterable iterable) { + if (iterable instanceof CloseableIterable) { + return (CloseableIterable) iterable; + } else if (iterable instanceof Closeable) { + Closeable asCloseable = (Closeable) iterable; + return combine(iterable, asCloseable); + } else { + return withNoopClose(iterable); + } + } + /** * Returns a closeable iterator over elements of type {@code T}. * diff --git a/api/src/test/java/org/apache/iceberg/util/RandomUtil.java b/api/src/test/java/org/apache/iceberg/util/RandomUtil.java index 9131e6166133..42dd6825a4d5 100644 --- a/api/src/test/java/org/apache/iceberg/util/RandomUtil.java +++ b/api/src/test/java/org/apache/iceberg/util/RandomUtil.java @@ -200,7 +200,10 @@ public static Object generateDictionaryEncodablePrimitive( "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; private static String randomString(Random random) { - int length = random.nextInt(50); + return generateString(random.nextInt(50), random); + } + + public static String generateString(int length, Random random) { byte[] buffer = new byte[length]; for (int i = 0; i < length; i += 1) { diff --git a/core/src/main/java/org/apache/iceberg/util/SortedMerge.java b/core/src/main/java/org/apache/iceberg/util/SortedMerge.java index d93116852eb9..62bc89bae96f 100644 --- a/core/src/main/java/org/apache/iceberg/util/SortedMerge.java +++ b/core/src/main/java/org/apache/iceberg/util/SortedMerge.java @@ -21,6 +21,7 @@ import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Arrays; import java.util.Comparator; import java.util.Iterator; import java.util.List; @@ -30,6 +31,7 @@ import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; /** * An Iterable that merges the items from other Iterables in order. @@ -39,6 +41,17 @@ * @param the type of objects produced by this Iterable */ public class SortedMerge extends CloseableGroup implements CloseableIterable { + public static > CloseableIterable of( + Iterable left, Iterable right) { + return of(Arrays.asList(left, right)); + } + + public static > CloseableIterable of(List> iterables) { + List> closeableIterables = + Lists.transform(iterables, CloseableIterable::of); + return new SortedMerge<>(Comparator.naturalOrder(), closeableIterables); + } + private final Comparator comparator; private final List> iterables; diff --git a/core/src/main/java/org/apache/iceberg/variants/PrimitiveWrapper.java b/core/src/main/java/org/apache/iceberg/variants/PrimitiveWrapper.java new file mode 100644 index 000000000000..96d6229cbd27 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/PrimitiveWrapper.java @@ -0,0 +1,206 @@ +/* + * 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.variants; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.variants.Variants.Primitives; + +class PrimitiveWrapper implements VariantPrimitive { + private static final byte NULL_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_NULL); + private static final byte TRUE_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_TRUE); + private static final byte FALSE_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_FALSE); + private static final byte INT8_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_INT8); + private static final byte INT16_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_INT16); + private static final byte INT32_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_INT32); + private static final byte INT64_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_INT64); + private static final byte FLOAT_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_FLOAT); + private static final byte DOUBLE_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_DOUBLE); + private static final byte DATE_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_DATE); + private static final byte TIMESTAMPTZ_HEADER = + VariantUtil.primitiveHeader(Primitives.TYPE_TIMESTAMPTZ); + private static final byte TIMESTAMPNTZ_HEADER = + VariantUtil.primitiveHeader(Primitives.TYPE_TIMESTAMPNTZ); + private static final byte DECIMAL4_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_DECIMAL4); + private static final byte DECIMAL8_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_DECIMAL8); + private static final byte DECIMAL16_HEADER = + VariantUtil.primitiveHeader(Primitives.TYPE_DECIMAL16); + private static final byte BINARY_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_BINARY); + private static final byte STRING_HEADER = VariantUtil.primitiveHeader(Primitives.TYPE_STRING); + + private final Variants.PhysicalType type; + private final T value; + private ByteBuffer buffer = null; + + PrimitiveWrapper(Variants.PhysicalType type, T value) { + this.type = type; + this.value = value; + } + + @Override + public Variants.PhysicalType type() { + return type; + } + + @Override + public T get() { + return value; + } + + @Override + public int sizeInBytes() { + switch (type()) { + case NULL: + case BOOLEAN_TRUE: + case BOOLEAN_FALSE: + return 1; // 1 header only + case INT8: + return 2; // 1 header + 1 value + case INT16: + return 3; // 1 header + 2 value + case INT32: + case DATE: + case FLOAT: + return 5; // 1 header + 4 value + case INT64: + case DOUBLE: + case TIMESTAMPTZ: + case TIMESTAMPNTZ: + return 9; // 1 header + 8 value + case DECIMAL4: + return 6; // 1 header + 1 scale + 4 unscaled value + case DECIMAL8: + return 10; // 1 header + 1 scale + 8 unscaled value + case DECIMAL16: + return 18; // 1 header + 1 scale + 16 unscaled value + case BINARY: + return 5 + ((ByteBuffer) value).remaining(); // 1 header + 4 length + value length + case STRING: + if (null == buffer) { + this.buffer = ByteBuffer.wrap(((String) value).getBytes(StandardCharsets.UTF_8)); + } + + return 5 + buffer.remaining(); // 1 header + 4 length + value length + } + + throw new UnsupportedOperationException("Unsupported primitive type: " + type()); + } + + @Override + public int writeTo(ByteBuffer outBuffer, int offset) { + Preconditions.checkArgument( + outBuffer.order() == ByteOrder.LITTLE_ENDIAN, "Invalid byte order: big endian"); + switch (type()) { + case NULL: + outBuffer.put(offset, NULL_HEADER); + return 1; + case BOOLEAN_TRUE: + outBuffer.put(offset, TRUE_HEADER); + return 1; + case BOOLEAN_FALSE: + outBuffer.put(offset, FALSE_HEADER); + return 1; + case INT8: + outBuffer.put(offset, INT8_HEADER); + outBuffer.put(offset + 1, (Byte) value); + return 2; + case INT16: + outBuffer.put(offset, INT16_HEADER); + outBuffer.putShort(offset + 1, (Short) value); + return 3; + case INT32: + outBuffer.put(offset, INT32_HEADER); + outBuffer.putInt(offset + 1, (Integer) value); + return 5; + case INT64: + outBuffer.put(offset, INT64_HEADER); + outBuffer.putLong(offset + 1, (Long) value); + return 9; + case FLOAT: + outBuffer.put(offset, FLOAT_HEADER); + outBuffer.putFloat(offset + 1, (Float) value); + return 5; + case DOUBLE: + outBuffer.put(offset, DOUBLE_HEADER); + outBuffer.putDouble(offset + 1, (Double) value); + return 9; + case DATE: + outBuffer.put(offset, DATE_HEADER); + outBuffer.putInt(offset + 1, (Integer) value); + return 5; + case TIMESTAMPTZ: + outBuffer.put(offset, TIMESTAMPTZ_HEADER); + outBuffer.putLong(offset + 1, (Long) value); + return 9; + case TIMESTAMPNTZ: + outBuffer.put(offset, TIMESTAMPNTZ_HEADER); + outBuffer.putLong(offset + 1, (Long) value); + return 9; + case DECIMAL4: + BigDecimal decimal4 = (BigDecimal) value; + outBuffer.put(offset, DECIMAL4_HEADER); + outBuffer.put(offset + 1, (byte) decimal4.scale()); + outBuffer.putInt(offset + 2, decimal4.unscaledValue().intValueExact()); + return 6; + case DECIMAL8: + BigDecimal decimal8 = (BigDecimal) value; + outBuffer.put(offset, DECIMAL8_HEADER); + outBuffer.put(offset + 1, (byte) decimal8.scale()); + outBuffer.putLong(offset + 2, decimal8.unscaledValue().longValueExact()); + return 10; + case DECIMAL16: + BigDecimal decimal16 = (BigDecimal) value; + byte padding = (byte) (decimal16.signum() < 0 ? 0xFF : 0x00); + byte[] bytes = decimal16.unscaledValue().toByteArray(); + outBuffer.put(offset, DECIMAL16_HEADER); + outBuffer.put(offset + 1, (byte) decimal16.scale()); + for (int i = 0; i < 16; i += 1) { + if (i < bytes.length) { + // copy the big endian value and convert to little endian + outBuffer.put(offset + 2 + i, bytes[bytes.length - i - 1]); + } else { + // pad with 0x00 or 0xFF depending on the sign + outBuffer.put(offset + 2 + i, padding); + } + } + return 18; + case BINARY: + ByteBuffer binary = (ByteBuffer) value; + outBuffer.put(offset, BINARY_HEADER); + outBuffer.putInt(offset + 1, binary.remaining()); + VariantUtil.writeBufferAbsolute(outBuffer, offset + 5, binary); + return 5 + binary.remaining(); + case STRING: + // TODO: use short string when possible + if (null == buffer) { + this.buffer = ByteBuffer.wrap(((String) value).getBytes(StandardCharsets.UTF_8)); + } + + outBuffer.put(offset, STRING_HEADER); + outBuffer.putInt(offset + 1, buffer.remaining()); + VariantUtil.writeBufferAbsolute(outBuffer, offset + 5, buffer); + return 5 + buffer.remaining(); + } + + throw new UnsupportedOperationException("Unsupported primitive type: " + type()); + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/SerializedArray.java b/core/src/main/java/org/apache/iceberg/variants/SerializedArray.java new file mode 100644 index 000000000000..774553cbb4a3 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/SerializedArray.java @@ -0,0 +1,88 @@ +/* + * 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.variants; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class SerializedArray extends Variants.SerializedValue implements VariantArray { + private static final int OFFSET_SIZE_MASK = 0b1100; + private static final int OFFSET_SIZE_SHIFT = 2; + private static final int IS_LARGE = 0b10000; + + @VisibleForTesting + static SerializedArray from(SerializedMetadata metadata, byte[] bytes) { + return from(metadata, ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN), bytes[0]); + } + + static SerializedArray from(SerializedMetadata metadata, ByteBuffer value, int header) { + Preconditions.checkArgument( + value.order() == ByteOrder.LITTLE_ENDIAN, "Unsupported byte order: big endian"); + Variants.BasicType basicType = VariantUtil.basicType(header); + Preconditions.checkArgument( + basicType == Variants.BasicType.ARRAY, "Invalid array, basic type: " + basicType); + return new SerializedArray(metadata, value, header); + } + + private final SerializedMetadata metadata; + private final ByteBuffer value; + private final int offsetSize; + private final int offsetListOffset; + private final int dataOffset; + private final VariantValue[] array; + + private SerializedArray(SerializedMetadata metadata, ByteBuffer value, int header) { + this.metadata = metadata; + this.value = value; + this.offsetSize = 1 + ((header & OFFSET_SIZE_MASK) >> OFFSET_SIZE_SHIFT); + int numElementsSize = ((header & IS_LARGE) == IS_LARGE) ? 4 : 1; + int numElements = + VariantUtil.readLittleEndianUnsigned(value, Variants.HEADER_SIZE, numElementsSize); + this.offsetListOffset = Variants.HEADER_SIZE + numElementsSize; + this.dataOffset = offsetListOffset + ((1 + numElements) * offsetSize); + this.array = new VariantValue[numElements]; + } + + @VisibleForTesting + int numElements() { + return array.length; + } + + @Override + public VariantValue get(int index) { + if (null == array[index]) { + int offset = + VariantUtil.readLittleEndianUnsigned( + value, offsetListOffset + (offsetSize * index), offsetSize); + int next = + VariantUtil.readLittleEndianUnsigned( + value, offsetListOffset + (offsetSize * (1 + index)), offsetSize); + array[index] = + Variants.from(metadata, VariantUtil.slice(value, dataOffset + offset, next - offset)); + } + return array[index]; + } + + @Override + public ByteBuffer buffer() { + return value; + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/SerializedMetadata.java b/core/src/main/java/org/apache/iceberg/variants/SerializedMetadata.java new file mode 100644 index 000000000000..30f4903db281 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/SerializedMetadata.java @@ -0,0 +1,113 @@ +/* + * 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.variants; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class SerializedMetadata implements VariantMetadata, Variants.Serialized { + private static final int SUPPORTED_VERSION = 1; + private static final int VERSION_MASK = 0b1111; + private static final int SORTED_STRINGS = 0b10000; + private static final int OFFSET_SIZE_MASK = 0b11000000; + private static final int OFFSET_SIZE_SHIFT = 6; + + static final ByteBuffer EMPTY_V1_BUFFER = + ByteBuffer.wrap(new byte[] {0x01, 0x00}).order(ByteOrder.LITTLE_ENDIAN); + + static SerializedMetadata from(byte[] bytes) { + return from(ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN)); + } + + static SerializedMetadata from(ByteBuffer metadata) { + Preconditions.checkArgument( + metadata.order() == ByteOrder.LITTLE_ENDIAN, "Unsupported byte order: big endian"); + int header = VariantUtil.readByte(metadata, 0); + int version = header & VERSION_MASK; + Preconditions.checkArgument(SUPPORTED_VERSION == version, "Unsupported version: %s", version); + return new SerializedMetadata(metadata, header); + } + + private final ByteBuffer metadata; + private final boolean isSorted; + private final int offsetSize; + private final int offsetListOffset; + private final int dataOffset; + private final String[] dict; + + private SerializedMetadata(ByteBuffer metadata, int header) { + this.metadata = metadata; + this.isSorted = (header & SORTED_STRINGS) == SORTED_STRINGS; + this.offsetSize = 1 + ((header & OFFSET_SIZE_MASK) >> OFFSET_SIZE_SHIFT); + int dictSize = VariantUtil.readLittleEndianUnsigned(metadata, Variants.HEADER_SIZE, offsetSize); + this.dict = new String[dictSize]; + this.offsetListOffset = Variants.HEADER_SIZE + offsetSize; + this.dataOffset = offsetListOffset + ((1 + dictSize) * offsetSize); + } + + @VisibleForTesting + int dictionarySize() { + return dict.length; + } + + @VisibleForTesting + boolean isSorted() { + return isSorted; + } + + /** Returns the position of the string in the metadata, or -1 if the string is not found. */ + @Override + public int id(String name) { + if (name != null) { + if (isSorted) { + return VariantUtil.find(dict.length, name, this::get); + } else { + for (int id = 0; id < dict.length; id += 1) { + if (name.equals(get(id))) { + return id; + } + } + } + } + + return -1; + } + + /** Returns the string for the given dictionary id. */ + @Override + public String get(int index) { + if (null == dict[index]) { + int offset = + VariantUtil.readLittleEndianUnsigned( + metadata, offsetListOffset + (offsetSize * index), offsetSize); + int next = + VariantUtil.readLittleEndianUnsigned( + metadata, offsetListOffset + (offsetSize * (1 + index)), offsetSize); + dict[index] = VariantUtil.readString(metadata, dataOffset + offset, next - offset); + } + return dict[index]; + } + + @Override + public ByteBuffer buffer() { + return metadata; + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/SerializedObject.java b/core/src/main/java/org/apache/iceberg/variants/SerializedObject.java new file mode 100644 index 000000000000..6eb103959420 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/SerializedObject.java @@ -0,0 +1,229 @@ +/* + * 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.variants; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; + +class SerializedObject extends Variants.SerializedValue implements VariantObject { + private static final int OFFSET_SIZE_MASK = 0b1100; + private static final int OFFSET_SIZE_SHIFT = 2; + private static final int FIELD_ID_SIZE_MASK = 0b110000; + private static final int FIELD_ID_SIZE_SHIFT = 4; + private static final int IS_LARGE = 0b1000000; + + static SerializedObject from(SerializedMetadata metadata, byte[] bytes) { + return from(metadata, ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN), bytes[0]); + } + + static SerializedObject from(SerializedMetadata metadata, ByteBuffer value, int header) { + Preconditions.checkArgument( + value.order() == ByteOrder.LITTLE_ENDIAN, "Unsupported byte order: big endian"); + Variants.BasicType basicType = VariantUtil.basicType(header); + Preconditions.checkArgument( + basicType == Variants.BasicType.OBJECT, "Invalid object, basic type: " + basicType); + return new SerializedObject(metadata, value, header); + } + + private final SerializedMetadata metadata; + private final ByteBuffer value; + private final int fieldIdSize; + private final int fieldIdListOffset; + private final Integer[] fieldIds; + private final int offsetSize; + private final int offsetListOffset; + private final int[] offsets; + private final int[] lengths; + private final int dataOffset; + private final VariantValue[] values; + + private SerializedObject(SerializedMetadata metadata, ByteBuffer value, int header) { + this.metadata = metadata; + this.value = value; + this.offsetSize = 1 + ((header & OFFSET_SIZE_MASK) >> OFFSET_SIZE_SHIFT); + this.fieldIdSize = 1 + ((header & FIELD_ID_SIZE_MASK) >> FIELD_ID_SIZE_SHIFT); + int numElementsSize = ((header & IS_LARGE) == IS_LARGE) ? 4 : 1; + int numElements = + VariantUtil.readLittleEndianUnsigned(value, Variants.HEADER_SIZE, numElementsSize); + this.fieldIdListOffset = Variants.HEADER_SIZE + numElementsSize; + this.fieldIds = new Integer[numElements]; + this.offsetListOffset = fieldIdListOffset + (numElements * fieldIdSize); + this.offsets = new int[numElements]; + this.lengths = new int[numElements]; + this.dataOffset = offsetListOffset + ((1 + numElements) * offsetSize); + this.values = new VariantValue[numElements]; + + if (numElements > 0) { + initOffsetsAndLengths(numElements); + } + } + + private void initOffsetsAndLengths(int numElements) { + // populate offsets list + Map offsetToLength = Maps.newHashMap(); + for (int index = 0; index < numElements; index += 1) { + offsets[index] = + VariantUtil.readLittleEndianUnsigned( + value, offsetListOffset + (index * offsetSize), offsetSize); + + offsetToLength.put(offsets[index], 0); + } + + int dataLength = + VariantUtil.readLittleEndianUnsigned( + value, offsetListOffset + (numElements * offsetSize), offsetSize); + offsetToLength.put(dataLength, 0); + + // populate lengths list by sorting offsets + List sortedOffsets = + offsetToLength.keySet().stream().sorted().collect(Collectors.toList()); + for (int index = 0; index < numElements; index += 1) { + int offset = sortedOffsets.get(index); + int length = sortedOffsets.get(index + 1) - offset; + offsetToLength.put(offset, length); + } + + for (int index = 0; index < lengths.length; index += 1) { + lengths[index] = offsetToLength.get(offsets[index]); + } + } + + @VisibleForTesting + int numElements() { + return fieldIds.length; + } + + SerializedMetadata metadata() { + return metadata; + } + + Iterable> fields() { + return () -> + new Iterator<>() { + private int index = 0; + + @Override + public boolean hasNext() { + return index < fieldIds.length; + } + + @Override + public Pair next() { + Pair next = Pair.of(metadata.get(id(index)), index); + index += 1; + return next; + } + }; + } + + public Iterable fieldNames() { + return () -> + new Iterator<>() { + private int index = 0; + + @Override + public boolean hasNext() { + return index < fieldIds.length; + } + + @Override + public String next() { + int id = id(index); + index += 1; + return metadata.get(id); + } + }; + } + + private int id(int index) { + if (null == fieldIds[index]) { + fieldIds[index] = + VariantUtil.readLittleEndianUnsigned( + value, fieldIdListOffset + (index * fieldIdSize), fieldIdSize); + } + + return fieldIds[index]; + } + + @Override + public VariantValue get(String name) { + // keys are ordered lexicographically by the name + int index = VariantUtil.find(fieldIds.length, name, pos -> metadata.get(id(pos))); + + if (index < 0) { + return null; + } + + if (null == values[index]) { + values[index] = + Variants.from( + metadata, VariantUtil.slice(value, dataOffset + offsets[index], lengths[index])); + } + + return values[index]; + } + + /** + * Retrieve a field value as a ByteBuffer. + * + * @param name field name + * @return the field value as a ByteBuffer + */ + ByteBuffer sliceValue(String name) { + int index = VariantUtil.find(fieldIds.length, name, pos -> metadata.get(id(pos))); + + if (index < 0) { + return null; + } + + return sliceValue(index); + } + + /** + * Retrieve a field value as a ByteBuffer. + * + * @param index field index within the object + * @return the field value as a ByteBuffer + */ + ByteBuffer sliceValue(int index) { + if (values[index] != null) { + return ((Variants.Serialized) values[index]).buffer(); + } + + return VariantUtil.slice(value, dataOffset + offsets[index], lengths[index]); + } + + @Override + public ByteBuffer buffer() { + return value; + } + + @Override + public int sizeInBytes() { + return value.remaining(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/SerializedPrimitive.java b/core/src/main/java/org/apache/iceberg/variants/SerializedPrimitive.java new file mode 100644 index 000000000000..1a6bd37a4ff3 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/SerializedPrimitive.java @@ -0,0 +1,130 @@ +/* + * 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.variants; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class SerializedPrimitive extends Variants.SerializedValue implements VariantPrimitive { + private static final int PRIMITIVE_TYPE_SHIFT = 2; + private static final int PRIMITIVE_OFFSET = Variants.HEADER_SIZE; + + static SerializedPrimitive from(byte[] bytes) { + return from(ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN), bytes[0]); + } + + static SerializedPrimitive from(ByteBuffer value, int header) { + Preconditions.checkArgument( + value.order() == ByteOrder.LITTLE_ENDIAN, "Unsupported byte order: big endian"); + Variants.BasicType basicType = VariantUtil.basicType(header); + Preconditions.checkArgument( + basicType == Variants.BasicType.PRIMITIVE, + "Invalid primitive, basic type != PRIMITIVE: " + basicType); + return new SerializedPrimitive(value, header); + } + + private final ByteBuffer value; + private final Variants.PhysicalType type; + private Object primitive = null; + + private SerializedPrimitive(ByteBuffer value, int header) { + this.value = value; + this.type = Variants.PhysicalType.from(header >> PRIMITIVE_TYPE_SHIFT); + } + + private Object read() { + switch (type) { + case NULL: + return null; + case BOOLEAN_TRUE: + return true; + case BOOLEAN_FALSE: + return false; + case INT8: + return VariantUtil.readLittleEndianInt8(value, PRIMITIVE_OFFSET); + case INT16: + return VariantUtil.readLittleEndianInt16(value, PRIMITIVE_OFFSET); + case INT32: + case DATE: + return VariantUtil.readLittleEndianInt32(value, PRIMITIVE_OFFSET); + case INT64: + case TIMESTAMPTZ: + case TIMESTAMPNTZ: + return VariantUtil.readLittleEndianInt64(value, PRIMITIVE_OFFSET); + case FLOAT: + return VariantUtil.readFloat(value, PRIMITIVE_OFFSET); + case DOUBLE: + return VariantUtil.readDouble(value, PRIMITIVE_OFFSET); + case DECIMAL4: + { + int scale = VariantUtil.readByte(value, PRIMITIVE_OFFSET); + int unscaled = VariantUtil.readLittleEndianInt32(value, PRIMITIVE_OFFSET + 1); + return new BigDecimal(BigInteger.valueOf(unscaled), scale); + } + case DECIMAL8: + { + int scale = VariantUtil.readByte(value, PRIMITIVE_OFFSET); + long unscaled = VariantUtil.readLittleEndianInt64(value, PRIMITIVE_OFFSET + 1); + return new BigDecimal(BigInteger.valueOf(unscaled), scale); + } + case DECIMAL16: + { + int scale = VariantUtil.readByte(value, PRIMITIVE_OFFSET); + byte[] unscaled = new byte[16]; + for (int i = 0; i < 16; i += 1) { + unscaled[i] = (byte) VariantUtil.readByte(value, PRIMITIVE_OFFSET + 16 - i); + } + return new BigDecimal(new BigInteger(unscaled), scale); + } + case BINARY: + { + int size = VariantUtil.readLittleEndianInt32(value, PRIMITIVE_OFFSET); + return VariantUtil.slice(value, PRIMITIVE_OFFSET + 4, size); + } + case STRING: + { + int size = VariantUtil.readLittleEndianInt32(value, PRIMITIVE_OFFSET); + return VariantUtil.readString(value, PRIMITIVE_OFFSET + 4, size); + } + } + + throw new UnsupportedOperationException("Unsupported primitive type: " + type); + } + + @Override + public Variants.PhysicalType type() { + return type; + } + + @Override + public Object get() { + if (null == primitive) { + this.primitive = read(); + } + return primitive; + } + + @Override + public ByteBuffer buffer() { + return value; + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/SerializedShortString.java b/core/src/main/java/org/apache/iceberg/variants/SerializedShortString.java new file mode 100644 index 000000000000..3004a075def1 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/SerializedShortString.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.variants; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class SerializedShortString extends Variants.SerializedValue implements VariantPrimitive { + private static final int LENGTH_MASK = 0b11111100; + private static final int LENGTH_SHIFT = 2; + + static SerializedShortString from(byte[] bytes) { + return from(ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN), bytes[0]); + } + + static SerializedShortString from(ByteBuffer value, int header) { + Preconditions.checkArgument( + value.order() == ByteOrder.LITTLE_ENDIAN, "Unsupported byte order: big endian"); + Variants.BasicType basicType = VariantUtil.basicType(header); + Preconditions.checkArgument( + basicType == Variants.BasicType.SHORT_STRING, + "Invalid short string, basic type: " + basicType); + return new SerializedShortString(value, header); + } + + private final ByteBuffer value; + private final int length; + private String string = null; + + private SerializedShortString(ByteBuffer value, int header) { + this.value = value; + this.length = ((header & LENGTH_MASK) >> LENGTH_SHIFT); + } + + @Override + public Variants.PhysicalType type() { + return Variants.PhysicalType.STRING; + } + + @Override + public String get() { + if (null == string) { + this.string = VariantUtil.readString(value, Variants.HEADER_SIZE, length); + } + return string; + } + + @Override + public ByteBuffer buffer() { + return value; + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java b/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java new file mode 100644 index 000000000000..e9e734fce0dd --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java @@ -0,0 +1,211 @@ +/* + * 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.variants; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Map; +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.Maps; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.SortedMerge; + +/** + * A variant Object that handles full or partial shredding. + * + *

    Metadata stored for an object must be the same regardless of whether the object is shredded. + * This class assumes that the metadata from the unshredded object can be used for the shredded + * fields. This also does not allow updating or replacing the metadata for the unshredded object, + * which could require recursively rewriting field IDs. + */ +class ShreddedObject implements VariantObject { + private final SerializedMetadata metadata; + private final SerializedObject unshredded; + private final Map shreddedFields = Maps.newHashMap(); + private SerializationState serializationState = null; + + ShreddedObject(SerializedMetadata metadata) { + this.metadata = metadata; + this.unshredded = null; + } + + ShreddedObject(SerializedObject unshredded) { + this.metadata = unshredded.metadata(); + this.unshredded = unshredded; + } + + public void put(String field, VariantValue value) { + Preconditions.checkArgument( + metadata.id(field) >= 0, "Cannot find field name in metadata: %s", field); + + // allow setting fields that are contained in unshredded. this avoids read-time failures and + // simplifies replacing field values. + shreddedFields.put(field, value); + this.serializationState = null; + } + + @Override + public VariantValue get(String field) { + // the shredded value takes precedence if there is a conflict + VariantValue value = shreddedFields.get(field); + if (value != null) { + return value; + } + + if (unshredded != null) { + return unshredded.get(field); + } + + return null; + } + + @Override + public int sizeInBytes() { + if (null == serializationState) { + this.serializationState = new SerializationState(metadata, unshredded, shreddedFields); + } + + return serializationState.size(); + } + + @Override + public int writeTo(ByteBuffer buffer, int offset) { + Preconditions.checkArgument( + buffer.order() == ByteOrder.LITTLE_ENDIAN, "Invalid byte order: big endian"); + + if (null == serializationState) { + this.serializationState = new SerializationState(metadata, unshredded, shreddedFields); + } + + return serializationState.writeTo(buffer, offset); + } + + /** Common state for {@link #size()} and {@link #writeTo(ByteBuffer, int)} */ + private static class SerializationState { + private final SerializedMetadata metadata; + private final Map unshreddedFields; + private final Map shreddedFields; + private final int dataSize; + private final int numElements; + private final boolean isLarge; + private final int fieldIdSize; + private final int offsetSize; + + private SerializationState( + SerializedMetadata metadata, + SerializedObject unshredded, + Map shreddedFields) { + this.metadata = metadata; + // field ID size is the size needed to store the largest field ID in the data + this.fieldIdSize = VariantUtil.sizeOf(metadata.dictionarySize()); + this.shreddedFields = shreddedFields; + + int totalDataSize = 0; + // get the unshredded field names and values as byte buffers + ImmutableMap.Builder unshreddedBuilder = ImmutableMap.builder(); + if (unshredded != null) { + for (Pair field : unshredded.fields()) { + // if the value is replaced by an unshredded field, don't include it + String name = field.first(); + boolean replaced = shreddedFields.containsKey(name); + if (!replaced) { + ByteBuffer value = unshredded.sliceValue(field.second()); + unshreddedBuilder.put(name, value); + totalDataSize += value.remaining(); + } + } + } + + this.unshreddedFields = unshreddedBuilder.build(); + // duplicates are suppressed when creating unshreddedFields + this.numElements = unshreddedFields.size() + shreddedFields.size(); + // object is large if the number of elements can't be stored in 1 byte + this.isLarge = numElements > 0xFF; + + for (VariantValue value : shreddedFields.values()) { + totalDataSize += value.sizeInBytes(); + } + + this.dataSize = totalDataSize; + // offset size is the size needed to store the length of the data section + this.offsetSize = VariantUtil.sizeOf(totalDataSize); + } + + private int size() { + return 1 /* header */ + + (isLarge ? 4 : 1) /* num elements size */ + + numElements * fieldIdSize /* field ID list size */ + + (1 + numElements) * offsetSize /* offset list size */ + + dataSize; + } + + private int writeTo(ByteBuffer buffer, int offset) { + int fieldIdListOffset = + offset + 1 /* header size */ + (isLarge ? 4 : 1) /* num elements size */; + int offsetListOffset = fieldIdListOffset + (numElements * fieldIdSize); + int dataOffset = offsetListOffset + ((1 + numElements) * offsetSize); + byte header = VariantUtil.objectHeader(isLarge, fieldIdSize, offsetSize); + + VariantUtil.writeByte(buffer, header, offset); + VariantUtil.writeLittleEndianUnsigned(buffer, numElements, offset + 1, isLarge ? 4 : 1); + + // neither iterable is closeable, so it is okay to use Iterable + Iterable fields = + SortedMerge.of( + () -> unshreddedFields.keySet().stream().sorted().iterator(), + () -> shreddedFields.keySet().stream().sorted().iterator()); + + int nextValueOffset = 0; + int index = 0; + for (String field : fields) { + // write the field ID from the metadata dictionary + int id = metadata.id(field); + Preconditions.checkState(id >= 0, "Invalid metadata, missing: %s", field); + VariantUtil.writeLittleEndianUnsigned( + buffer, id, fieldIdListOffset + (index * fieldIdSize), fieldIdSize); + // write the data offset + VariantUtil.writeLittleEndianUnsigned( + buffer, nextValueOffset, offsetListOffset + (index * offsetSize), offsetSize); + + // copy or serialize the value into the data section + int valueSize; + VariantValue shreddedValue = shreddedFields.get(field); + if (shreddedValue != null) { + valueSize = shreddedValue.writeTo(buffer, dataOffset + nextValueOffset); + } else { + valueSize = + VariantUtil.writeBufferAbsolute( + buffer, dataOffset + nextValueOffset, unshreddedFields.get(field)); + } + + // update tracking + nextValueOffset += valueSize; + index += 1; + } + + // write the final size of the data section + VariantUtil.writeLittleEndianUnsigned( + buffer, nextValueOffset, offsetListOffset + (index * offsetSize), offsetSize); + + // return the total size + return (dataOffset - offset) + dataSize; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/Variant.java b/core/src/main/java/org/apache/iceberg/variants/Variant.java new file mode 100644 index 000000000000..b5606fa094b6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/Variant.java @@ -0,0 +1,28 @@ +/* + * 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.variants; + +/** A variant metadata and value pair. */ +public interface Variant { + /** Returns the metadata for all values in the variant. */ + VariantMetadata metadata(); + + /** Returns the variant value. */ + VariantValue value(); +} diff --git a/core/src/main/java/org/apache/iceberg/variants/VariantArray.java b/core/src/main/java/org/apache/iceberg/variants/VariantArray.java new file mode 100644 index 000000000000..55dbc071f15b --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/VariantArray.java @@ -0,0 +1,35 @@ +/* + * 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.variants; + +/** An variant array value. */ +public interface VariantArray extends VariantValue { + /** Returns the {@link VariantValue} at {@code index} in this array. */ + VariantValue get(int index); + + @Override + default Variants.PhysicalType type() { + return Variants.PhysicalType.ARRAY; + } + + @Override + default VariantArray asArray() { + return this; + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/VariantMetadata.java b/core/src/main/java/org/apache/iceberg/variants/VariantMetadata.java new file mode 100644 index 000000000000..91129e2328df --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/VariantMetadata.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.variants; + +import java.util.NoSuchElementException; + +/** A variant metadata dictionary. */ +public interface VariantMetadata extends Variants.Serialized { + /** Returns the ID for a {@code name} in the dictionary, or -1 if not present. */ + int id(String name); + + /** + * Returns the field name for an ID in metadata. + * + * @throws NoSuchElementException if the dictionary does not contain the ID + */ + String get(int id); +} diff --git a/core/src/main/java/org/apache/iceberg/variants/VariantObject.java b/core/src/main/java/org/apache/iceberg/variants/VariantObject.java new file mode 100644 index 000000000000..7bb82f94a467 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/VariantObject.java @@ -0,0 +1,35 @@ +/* + * 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.variants; + +/** An variant object value. */ +public interface VariantObject extends VariantValue { + /** Returns the {@link VariantValue} for the field named {@code name} in this object. */ + VariantValue get(String name); + + @Override + default Variants.PhysicalType type() { + return Variants.PhysicalType.OBJECT; + } + + @Override + default VariantObject asObject() { + return this; + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/VariantPrimitive.java b/core/src/main/java/org/apache/iceberg/variants/VariantPrimitive.java new file mode 100644 index 000000000000..73efb45ae91b --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/VariantPrimitive.java @@ -0,0 +1,29 @@ +/* + * 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.variants; + +/** A primitive variant value. */ +public interface VariantPrimitive extends VariantValue { + T get(); + + @Override + default VariantPrimitive asPrimitive() { + return this; + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/VariantUtil.java b/core/src/main/java/org/apache/iceberg/variants/VariantUtil.java new file mode 100644 index 000000000000..d6b78fe899e6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/VariantUtil.java @@ -0,0 +1,195 @@ +/* + * 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.variants; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import java.util.function.Function; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class VariantUtil { + private static final int BASIC_TYPE_MASK = 0b11; + private static final int BASIC_TYPE_PRIMITIVE = 0; + private static final int BASIC_TYPE_SHORT_STRING = 1; + private static final int BASIC_TYPE_OBJECT = 2; + private static final int BASIC_TYPE_ARRAY = 3; + + private VariantUtil() {} + + /** A hacky absolute put for ByteBuffer */ + static int writeBufferAbsolute(ByteBuffer buffer, int offset, ByteBuffer toCopy) { + int originalPosition = buffer.position(); + buffer.position(offset); + ByteBuffer copy = toCopy.duplicate(); + buffer.put(copy); // duplicate so toCopy is not modified + buffer.position(originalPosition); + Preconditions.checkArgument(copy.remaining() <= 0, "Not fully written"); + return toCopy.remaining(); + } + + static void writeByte(ByteBuffer buffer, int value, int offset) { + buffer.put(buffer.position() + offset, (byte) (value & 0xFF)); + } + + static void writeLittleEndianUnsigned(ByteBuffer buffer, int value, int offset, int size) { + int base = buffer.position() + offset; + switch (size) { + case 4: + buffer.putInt(base, value); + return; + case 3: + buffer.putShort(base, (short) (value & 0xFFFF)); + buffer.put(base + 2, (byte) ((value >> 16) & 0xFF)); + return; + case 2: + buffer.putShort(base, (short) (value & 0xFFFF)); + return; + case 1: + buffer.put(base, (byte) (value & 0xFF)); + return; + } + + throw new IllegalArgumentException("Invalid size: " + size); + } + + static byte readLittleEndianInt8(ByteBuffer buffer, int offset) { + return buffer.get(buffer.position() + offset); + } + + static short readLittleEndianInt16(ByteBuffer buffer, int offset) { + return buffer.getShort(buffer.position() + offset); + } + + static int readByte(ByteBuffer buffer, int offset) { + return buffer.get(buffer.position() + offset) & 0xFF; + } + + static int readLittleEndianUnsigned(ByteBuffer buffer, int offset, int size) { + int base = buffer.position() + offset; + switch (size) { + case 4: + return buffer.getInt(base); + case 3: + return (((int) buffer.getShort(base)) & 0xFFFF) | ((buffer.get(base + 2) & 0xFF) << 16); + case 2: + return ((int) buffer.getShort(base)) & 0xFFFF; + case 1: + return buffer.get(base) & 0xFF; + } + + throw new IllegalArgumentException("Invalid size: " + size); + } + + static int readLittleEndianInt32(ByteBuffer buffer, int offset) { + return buffer.getInt(buffer.position() + offset); + } + + static long readLittleEndianInt64(ByteBuffer buffer, int offset) { + return buffer.getLong(buffer.position() + offset); + } + + static float readFloat(ByteBuffer buffer, int offset) { + return buffer.getFloat(buffer.position() + offset); + } + + static double readDouble(ByteBuffer buffer, int offset) { + return buffer.getDouble(buffer.position() + offset); + } + + static ByteBuffer slice(ByteBuffer buffer, int offset, int length) { + ByteBuffer slice = buffer.duplicate(); + slice.order(ByteOrder.LITTLE_ENDIAN); + slice.position(buffer.position() + offset); + slice.limit(buffer.position() + offset + length); + return slice; + } + + static String readString(ByteBuffer buffer, int offset, int length) { + if (buffer.hasArray()) { + return new String( + buffer.array(), + buffer.arrayOffset() + buffer.position() + offset, + length, + StandardCharsets.UTF_8); + } else { + return StandardCharsets.UTF_8.decode(slice(buffer, offset, length)).toString(); + } + } + + static > int find(int size, T key, Function resolve) { + int low = 0; + int high = size - 1; + while (low <= high) { + int mid = (low + high) >>> 1; + T value = resolve.apply(mid); + int cmp = key.compareTo(value); + if (cmp == 0) { + return mid; + } else if (cmp < 0) { + high = mid - 1; + } else { + low = mid + 1; + } + } + + return -1; + } + + static int sizeOf(int maxValue) { + if (maxValue <= 0xFF) { + return 1; + } else if (maxValue <= 0xFFFF) { + return 2; + } else if (maxValue <= 0xFFFFFF) { + return 3; + } else { + return 4; + } + } + + static byte primitiveHeader(int primitiveType) { + return (byte) (primitiveType << Variants.Primitives.PRIMITIVE_TYPE_SHIFT); + } + + static byte objectHeader(boolean isLarge, int fieldIdSize, int offsetSize) { + return (byte) + ((isLarge ? 0b1000000 : 0) | ((fieldIdSize - 1) << 4) | ((offsetSize - 1) << 2) | 0b10); + } + + static byte arrayHeader(boolean isLarge, int offsetSize) { + return (byte) ((isLarge ? 0b10000 : 0) | (offsetSize - 1) << 2 | 0b11); + } + + static Variants.BasicType basicType(int header) { + int basicType = header & BASIC_TYPE_MASK; + switch (basicType) { + case BASIC_TYPE_PRIMITIVE: + return Variants.BasicType.PRIMITIVE; + case BASIC_TYPE_SHORT_STRING: + return Variants.BasicType.SHORT_STRING; + case BASIC_TYPE_OBJECT: + return Variants.BasicType.OBJECT; + case BASIC_TYPE_ARRAY: + return Variants.BasicType.ARRAY; + } + + throw new UnsupportedOperationException("Unsupported basic type: " + basicType); + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/VariantValue.java b/core/src/main/java/org/apache/iceberg/variants/VariantValue.java new file mode 100644 index 000000000000..26a43795f778 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/VariantValue.java @@ -0,0 +1,63 @@ +/* + * 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.variants; + +import java.nio.ByteBuffer; +import org.apache.iceberg.variants.Variants.PhysicalType; + +/** A variant value. */ +public interface VariantValue { + /** Returns the {@link PhysicalType} of this value. */ + PhysicalType type(); + + /** Returns the serialized size in bytes of this value. */ + int sizeInBytes(); + + /** + * Writes this value to the buffer at the given offset, ignoring the buffer's position and limit. + */ + int writeTo(ByteBuffer buffer, int offset); + + /** + * Returns this value as a {@link VariantPrimitive}. + * + * @throws IllegalArgumentException if the value is not a primitive + */ + default VariantPrimitive asPrimitive() { + throw new IllegalArgumentException("Not a primitive: " + this); + } + + /** + * Returns this value as a {@link VariantObject}. + * + * @throws IllegalArgumentException if the value is not an object + */ + default VariantObject asObject() { + throw new IllegalArgumentException("Not an object: " + this); + } + + /** + * Returns this value as a {@link VariantArray}. + * + * @throws IllegalArgumentException if the value is not an array + */ + default VariantArray asArray() { + throw new IllegalArgumentException("Not an array: " + this); + } +} diff --git a/core/src/main/java/org/apache/iceberg/variants/Variants.java b/core/src/main/java/org/apache/iceberg/variants/Variants.java new file mode 100644 index 000000000000..e8ea3d93ab77 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/variants/Variants.java @@ -0,0 +1,276 @@ +/* + * 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.variants; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.util.DateTimeUtil; + +public class Variants { + private Variants() {} + + enum LogicalType { + NULL, + BOOLEAN, + EXACT_NUMERIC, + FLOAT, + DOUBLE, + DATE, + TIMESTAMPTZ, + TIMESTAMPNTZ, + BINARY, + STRING, + ARRAY, + OBJECT + } + + public enum PhysicalType { + NULL(LogicalType.NULL, Void.class), + BOOLEAN_TRUE(LogicalType.BOOLEAN, Boolean.class), + BOOLEAN_FALSE(LogicalType.BOOLEAN, Boolean.class), + INT8(LogicalType.EXACT_NUMERIC, Byte.class), + INT16(LogicalType.EXACT_NUMERIC, Short.class), + INT32(LogicalType.EXACT_NUMERIC, Integer.class), + INT64(LogicalType.EXACT_NUMERIC, Long.class), + DOUBLE(LogicalType.DOUBLE, Double.class), + DECIMAL4(LogicalType.EXACT_NUMERIC, BigDecimal.class), + DECIMAL8(LogicalType.EXACT_NUMERIC, BigDecimal.class), + DECIMAL16(LogicalType.EXACT_NUMERIC, BigDecimal.class), + DATE(LogicalType.DATE, Integer.class), + TIMESTAMPTZ(LogicalType.TIMESTAMPTZ, Long.class), + TIMESTAMPNTZ(LogicalType.TIMESTAMPNTZ, Long.class), + FLOAT(LogicalType.FLOAT, Float.class), + BINARY(LogicalType.BINARY, ByteBuffer.class), + STRING(LogicalType.STRING, String.class), + ARRAY(LogicalType.ARRAY, List.class), + OBJECT(LogicalType.OBJECT, Map.class); + + private final LogicalType logicalType; + private final Class javaClass; + + PhysicalType(LogicalType logicalType, Class javaClass) { + this.logicalType = logicalType; + this.javaClass = javaClass; + } + + LogicalType toLogicalType() { + return logicalType; + } + + public Class javaClass() { + return javaClass; + } + + public static PhysicalType from(int primitiveType) { + switch (primitiveType) { + case Primitives.TYPE_NULL: + return NULL; + case Primitives.TYPE_TRUE: + return BOOLEAN_TRUE; + case Primitives.TYPE_FALSE: + return BOOLEAN_FALSE; + case Primitives.TYPE_INT8: + return INT8; + case Primitives.TYPE_INT16: + return INT16; + case Primitives.TYPE_INT32: + return INT32; + case Primitives.TYPE_INT64: + return INT64; + case Primitives.TYPE_DATE: + return DATE; + case Primitives.TYPE_TIMESTAMPTZ: + return TIMESTAMPTZ; + case Primitives.TYPE_TIMESTAMPNTZ: + return TIMESTAMPNTZ; + case Primitives.TYPE_FLOAT: + return FLOAT; + case Primitives.TYPE_DOUBLE: + return DOUBLE; + case Primitives.TYPE_DECIMAL4: + return DECIMAL4; + case Primitives.TYPE_DECIMAL8: + return DECIMAL8; + case Primitives.TYPE_DECIMAL16: + return DECIMAL16; + case Primitives.TYPE_BINARY: + return BINARY; + case Primitives.TYPE_STRING: + return STRING; + } + + throw new UnsupportedOperationException("Unknown primitive physical type: " + primitiveType); + } + } + + interface Serialized { + ByteBuffer buffer(); + } + + abstract static class SerializedValue implements VariantValue, Serialized { + @Override + public int sizeInBytes() { + return buffer().remaining(); + } + + @Override + public int writeTo(ByteBuffer buffer, int offset) { + ByteBuffer value = buffer(); + VariantUtil.writeBufferAbsolute(buffer, offset, value); + return value.remaining(); + } + } + + static class Primitives { + static final int TYPE_NULL = 0; + static final int TYPE_TRUE = 1; + static final int TYPE_FALSE = 2; + static final int TYPE_INT8 = 3; + static final int TYPE_INT16 = 4; + static final int TYPE_INT32 = 5; + static final int TYPE_INT64 = 6; + static final int TYPE_DOUBLE = 7; + static final int TYPE_DECIMAL4 = 8; + static final int TYPE_DECIMAL8 = 9; + static final int TYPE_DECIMAL16 = 10; + static final int TYPE_DATE = 11; + static final int TYPE_TIMESTAMPTZ = 12; // equivalent to timestamptz + static final int TYPE_TIMESTAMPNTZ = 13; // equivalent to timestamp + static final int TYPE_FLOAT = 14; + static final int TYPE_BINARY = 15; + static final int TYPE_STRING = 16; + + static final int PRIMITIVE_TYPE_SHIFT = 2; + + private Primitives() {} + } + + static final int HEADER_SIZE = 1; + + enum BasicType { + PRIMITIVE, + SHORT_STRING, + OBJECT, + ARRAY + } + + public static VariantValue from(ByteBuffer metadata, ByteBuffer value) { + return from(SerializedMetadata.from(metadata), value); + } + + static VariantValue from(SerializedMetadata metadata, ByteBuffer value) { + int header = VariantUtil.readByte(value, 0); + BasicType basicType = VariantUtil.basicType(header); + switch (basicType) { + case PRIMITIVE: + return SerializedPrimitive.from(value, header); + case SHORT_STRING: + return SerializedShortString.from(value, header); + case OBJECT: + return SerializedObject.from(metadata, value, header); + case ARRAY: + return SerializedArray.from(metadata, value, header); + } + + throw new UnsupportedOperationException("Unsupported basic type: " + basicType); + } + + static VariantPrimitive ofNull() { + return new PrimitiveWrapper<>(PhysicalType.NULL, null); + } + + static VariantPrimitive of(boolean value) { + if (value) { + return new PrimitiveWrapper<>(PhysicalType.BOOLEAN_TRUE, true); + } else { + return new PrimitiveWrapper<>(PhysicalType.BOOLEAN_FALSE, false); + } + } + + static VariantPrimitive of(byte value) { + return new PrimitiveWrapper<>(PhysicalType.INT8, value); + } + + static VariantPrimitive of(short value) { + return new PrimitiveWrapper<>(PhysicalType.INT16, value); + } + + static VariantPrimitive of(int value) { + return new PrimitiveWrapper<>(PhysicalType.INT32, value); + } + + static VariantPrimitive of(long value) { + return new PrimitiveWrapper<>(PhysicalType.INT64, value); + } + + static VariantPrimitive of(float value) { + return new PrimitiveWrapper<>(PhysicalType.FLOAT, value); + } + + static VariantPrimitive of(double value) { + return new PrimitiveWrapper<>(PhysicalType.DOUBLE, value); + } + + static VariantPrimitive ofDate(int value) { + return new PrimitiveWrapper<>(PhysicalType.DATE, value); + } + + static VariantPrimitive ofIsoDate(String value) { + return ofDate(DateTimeUtil.isoDateToDays(value)); + } + + static VariantPrimitive ofTimestamptz(long value) { + return new PrimitiveWrapper<>(PhysicalType.TIMESTAMPTZ, value); + } + + static VariantPrimitive ofIsoTimestamptz(String value) { + return ofTimestamptz(DateTimeUtil.isoTimestamptzToMicros(value)); + } + + static VariantPrimitive ofTimestampntz(long value) { + return new PrimitiveWrapper<>(PhysicalType.TIMESTAMPNTZ, value); + } + + static VariantPrimitive ofIsoTimestampntz(String value) { + return ofTimestampntz(DateTimeUtil.isoTimestampToMicros(value)); + } + + static VariantPrimitive of(BigDecimal value) { + int bitLength = value.unscaledValue().bitLength(); + if (bitLength < 32) { + return new PrimitiveWrapper<>(PhysicalType.DECIMAL4, value); + } else if (bitLength < 64) { + return new PrimitiveWrapper<>(PhysicalType.DECIMAL8, value); + } else if (bitLength < 128) { + return new PrimitiveWrapper<>(PhysicalType.DECIMAL16, value); + } + + throw new UnsupportedOperationException("Unsupported decimal precision: " + value.precision()); + } + + static VariantPrimitive of(ByteBuffer value) { + return new PrimitiveWrapper<>(PhysicalType.BINARY, value); + } + + static VariantPrimitive of(String value) { + return new PrimitiveWrapper<>(PhysicalType.STRING, value); + } +} diff --git a/core/src/test/java/org/apache/iceberg/variants/TestPrimitiveWrapper.java b/core/src/test/java/org/apache/iceberg/variants/TestPrimitiveWrapper.java new file mode 100644 index 000000000000..fd113f9cece3 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/variants/TestPrimitiveWrapper.java @@ -0,0 +1,82 @@ +/* + * 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.variants; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.FieldSource; + +public class TestPrimitiveWrapper { + private static final VariantPrimitive[] PRIMITIVES = + new VariantPrimitive[] { + Variants.ofNull(), + Variants.of(true), + Variants.of(false), + Variants.of((byte) 34), + Variants.of((byte) -34), + Variants.of((short) 1234), + Variants.of((short) -1234), + Variants.of(12345), + Variants.of(-12345), + Variants.of(9876543210L), + Variants.of(-9876543210L), + Variants.of(10.11F), + Variants.of(-10.11F), + Variants.of(14.3D), + Variants.of(-14.3D), + Variants.ofIsoDate("2024-11-07"), + Variants.ofIsoDate("1957-11-07"), + Variants.ofIsoTimestamptz("2024-11-07T12:33:54.123456+00:00"), + Variants.ofIsoTimestamptz("1957-11-07T12:33:54.123456+00:00"), + Variants.ofIsoTimestampntz("2024-11-07T12:33:54.123456"), + Variants.ofIsoTimestampntz("1957-11-07T12:33:54.123456"), + Variants.of(new BigDecimal("123456.7890")), // decimal4 + Variants.of(new BigDecimal("-123456.7890")), // decimal4 + Variants.of(new BigDecimal("1234567890.987654321")), // decimal8 + Variants.of(new BigDecimal("-1234567890.987654321")), // decimal8 + Variants.of(new BigDecimal("9876543210.123456789")), // decimal16 + Variants.of(new BigDecimal("-9876543210.123456789")), // decimal16 + Variants.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), + Variants.of("iceberg"), + }; + + @ParameterizedTest + @FieldSource("PRIMITIVES") + public void testPrimitiveValueSerialization(VariantPrimitive primitive) { + // write the value to the middle of a large buffer + int size = primitive.sizeInBytes(); + ByteBuffer buffer = ByteBuffer.allocate(size + 1000).order(ByteOrder.LITTLE_ENDIAN); + primitive.writeTo(buffer, 300); + + // create a copy that is limited to the value range + ByteBuffer readBuffer = buffer.duplicate().order(ByteOrder.LITTLE_ENDIAN); + readBuffer.position(300); + readBuffer.limit(300 + size); + + // read and validate the serialized bytes + VariantValue actual = Variants.from(SerializedMetadata.EMPTY_V1_BUFFER, readBuffer); + assertThat(actual.type()).isEqualTo(primitive.type()); + assertThat(actual).isInstanceOf(VariantPrimitive.class); + assertThat(actual.asPrimitive().get()).isEqualTo(primitive.get()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/variants/TestSerializedArray.java b/core/src/test/java/org/apache/iceberg/variants/TestSerializedArray.java new file mode 100644 index 000000000000..1e052572b85d --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/variants/TestSerializedArray.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.variants; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.nio.ByteBuffer; +import java.util.Random; +import org.apache.iceberg.util.RandomUtil; +import org.apache.iceberg.variants.Variants.PhysicalType; +import org.junit.jupiter.api.Test; + +public class TestSerializedArray { + private static final SerializedMetadata EMPTY_METADATA = + SerializedMetadata.from(SerializedMetadata.EMPTY_V1_BUFFER); + private static final SerializedPrimitive NULL = SerializedPrimitive.from(new byte[] {0x00}); + private static final SerializedPrimitive TRUE = SerializedPrimitive.from(new byte[] {0b100}); + private static final SerializedPrimitive FALSE = SerializedPrimitive.from(new byte[] {0b1000}); + private static final SerializedShortString STR = + SerializedShortString.from(new byte[] {0b11101, 'i', 'c', 'e', 'b', 'e', 'r', 'g'}); + private static final SerializedShortString A = + SerializedShortString.from(new byte[] {0b101, 'a'}); + private static final SerializedShortString B = + SerializedShortString.from(new byte[] {0b101, 'b'}); + private static final SerializedShortString C = + SerializedShortString.from(new byte[] {0b101, 'c'}); + private static final SerializedShortString D = + SerializedShortString.from(new byte[] {0b101, 'd'}); + private static final SerializedShortString E = + SerializedShortString.from(new byte[] {0b101, 'e'}); + private static final SerializedPrimitive I34 = SerializedPrimitive.from(new byte[] {0b1100, 34}); + private static final SerializedPrimitive I1234 = + SerializedPrimitive.from(new byte[] {0b10000, (byte) 0xD2, 0x04}); + private static final SerializedPrimitive DATE = + SerializedPrimitive.from(new byte[] {0b101100, (byte) 0xF4, 0x43, 0x00, 0x00}); + + private final Random random = new Random(374513); + + @Test + public void testEmptyArray() { + SerializedArray array = SerializedArray.from(EMPTY_METADATA, new byte[] {0b0011, 0x00}); + + assertThat(array.type()).isEqualTo(PhysicalType.ARRAY); + assertThat(array.numElements()).isEqualTo(0); + } + + @Test + public void testEmptyLargeArray() { + SerializedArray array = + SerializedArray.from(EMPTY_METADATA, new byte[] {0b10011, 0x00, 0x00, 0x00, 0x00}); + + assertThat(array.type()).isEqualTo(PhysicalType.ARRAY); + assertThat(array.numElements()).isEqualTo(0); + } + + @Test + public void testStringArray() { + ByteBuffer buffer = VariantTestUtil.createArray(A, B, C, D, E); + SerializedArray array = SerializedArray.from(EMPTY_METADATA, buffer, buffer.get(0)); + + assertThat(array.type()).isEqualTo(PhysicalType.ARRAY); + assertThat(array.numElements()).isEqualTo(5); + assertThat(array.get(0).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(0).asPrimitive().get()).isEqualTo("a"); + assertThat(array.get(1).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(1).asPrimitive().get()).isEqualTo("b"); + assertThat(array.get(2).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(2).asPrimitive().get()).isEqualTo("c"); + assertThat(array.get(3).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(3).asPrimitive().get()).isEqualTo("d"); + assertThat(array.get(4).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(4).asPrimitive().get()).isEqualTo("e"); + + assertThatThrownBy(() -> array.get(5)) + .isInstanceOf(ArrayIndexOutOfBoundsException.class) + .hasMessage("Index 5 out of bounds for length 5"); + } + + @Test + public void testStringDifferentLengths() { + ByteBuffer buffer = VariantTestUtil.createArray(A, B, C, STR, D, E); + SerializedArray array = SerializedArray.from(EMPTY_METADATA, buffer, buffer.get(0)); + + assertThat(array.type()).isEqualTo(PhysicalType.ARRAY); + assertThat(array.numElements()).isEqualTo(6); + assertThat(array.get(0).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(0).asPrimitive().get()).isEqualTo("a"); + assertThat(array.get(1).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(1).asPrimitive().get()).isEqualTo("b"); + assertThat(array.get(2).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(2).asPrimitive().get()).isEqualTo("c"); + assertThat(array.get(3).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(3).asPrimitive().get()).isEqualTo("iceberg"); + assertThat(array.get(4).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(4).asPrimitive().get()).isEqualTo("d"); + assertThat(array.get(5).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(5).asPrimitive().get()).isEqualTo("e"); + + assertThatThrownBy(() -> array.get(6)) + .isInstanceOf(ArrayIndexOutOfBoundsException.class) + .hasMessage("Index 6 out of bounds for length 6"); + } + + @Test + public void testArrayOfMixedTypes() { + ByteBuffer nestedBuffer = VariantTestUtil.createArray(A, C, D); + SerializedArray nested = + SerializedArray.from(EMPTY_METADATA, nestedBuffer, nestedBuffer.get(0)); + ByteBuffer buffer = + VariantTestUtil.createArray(DATE, I34, STR, NULL, E, B, FALSE, nested, TRUE, I1234); + SerializedArray array = SerializedArray.from(EMPTY_METADATA, buffer, buffer.get(0)); + + assertThat(array.type()).isEqualTo(PhysicalType.ARRAY); + assertThat(array.numElements()).isEqualTo(10); + assertThat(array.get(0).type()).isEqualTo(PhysicalType.DATE); + assertThat(array.get(0).asPrimitive().get()).isEqualTo(17396); + assertThat(array.get(1).type()).isEqualTo(PhysicalType.INT8); + assertThat(array.get(1).asPrimitive().get()).isEqualTo((byte) 34); + assertThat(array.get(2).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(2).asPrimitive().get()).isEqualTo("iceberg"); + assertThat(array.get(3).type()).isEqualTo(PhysicalType.NULL); + assertThat(array.get(3).asPrimitive().get()).isEqualTo(null); + assertThat(array.get(4).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(4).asPrimitive().get()).isEqualTo("e"); + assertThat(array.get(5).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(5).asPrimitive().get()).isEqualTo("b"); + assertThat(array.get(6).type()).isEqualTo(PhysicalType.BOOLEAN_FALSE); + assertThat(array.get(6).asPrimitive().get()).isEqualTo(false); + assertThat(array.get(8).type()).isEqualTo(PhysicalType.BOOLEAN_TRUE); + assertThat(array.get(8).asPrimitive().get()).isEqualTo(true); + assertThat(array.get(9).type()).isEqualTo(PhysicalType.INT16); + assertThat(array.get(9).asPrimitive().get()).isEqualTo((short) 1234); + + assertThatThrownBy(() -> array.get(10)) + .isInstanceOf(ArrayIndexOutOfBoundsException.class) + .hasMessage("Index 10 out of bounds for length 10"); + + assertThat(array.get(7).type()).isEqualTo(PhysicalType.ARRAY); + SerializedArray actualNested = (SerializedArray) array.get(7); + assertThat(actualNested.numElements()).isEqualTo(3); + assertThat(actualNested.get(0).type()).isEqualTo(PhysicalType.STRING); + assertThat(actualNested.get(0).asPrimitive().get()).isEqualTo("a"); + assertThat(actualNested.get(1).type()).isEqualTo(PhysicalType.STRING); + assertThat(actualNested.get(1).asPrimitive().get()).isEqualTo("c"); + assertThat(actualNested.get(2).type()).isEqualTo(PhysicalType.STRING); + assertThat(actualNested.get(2).asPrimitive().get()).isEqualTo("d"); + + assertThatThrownBy(() -> actualNested.get(3)) + .isInstanceOf(ArrayIndexOutOfBoundsException.class) + .hasMessage("Index 3 out of bounds for length 3"); + } + + @Test + public void testTwoByteOffsets() { + // a string larger than 255 bytes to push the value offset size above 1 byte + String randomString = RandomUtil.generateString(300, random); + SerializedPrimitive bigString = VariantTestUtil.createString(randomString); + + ByteBuffer buffer = VariantTestUtil.createArray(bigString, A, B, C); + SerializedArray array = SerializedArray.from(EMPTY_METADATA, buffer, buffer.get(0)); + + assertThat(array.type()).isEqualTo(PhysicalType.ARRAY); + assertThat(array.numElements()).isEqualTo(4); + assertThat(array.get(0).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(0).asPrimitive().get()).isEqualTo(randomString); + assertThat(array.get(1).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(1).asPrimitive().get()).isEqualTo("a"); + assertThat(array.get(2).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(2).asPrimitive().get()).isEqualTo("b"); + assertThat(array.get(3).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(3).asPrimitive().get()).isEqualTo("c"); + + assertThatThrownBy(() -> array.get(4)) + .isInstanceOf(ArrayIndexOutOfBoundsException.class) + .hasMessage("Index 4 out of bounds for length 4"); + } + + @Test + public void testThreeByteOffsets() { + // a string larger than 65535 bytes to push the value offset size above 1 byte + String randomString = RandomUtil.generateString(70_000, random); + SerializedPrimitive reallyBigString = VariantTestUtil.createString(randomString); + + ByteBuffer buffer = VariantTestUtil.createArray(reallyBigString, A, B, C); + SerializedArray array = SerializedArray.from(EMPTY_METADATA, buffer, buffer.get(0)); + + assertThat(array.type()).isEqualTo(PhysicalType.ARRAY); + assertThat(array.numElements()).isEqualTo(4); + assertThat(array.get(0).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(0).asPrimitive().get()).isEqualTo(randomString); + assertThat(array.get(1).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(1).asPrimitive().get()).isEqualTo("a"); + assertThat(array.get(2).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(2).asPrimitive().get()).isEqualTo("b"); + assertThat(array.get(3).type()).isEqualTo(PhysicalType.STRING); + assertThat(array.get(3).asPrimitive().get()).isEqualTo("c"); + + assertThatThrownBy(() -> array.get(4)) + .isInstanceOf(ArrayIndexOutOfBoundsException.class) + .hasMessage("Index 4 out of bounds for length 4"); + } + + @Test + public void testLargeArraySize() { + SerializedArray array = + SerializedArray.from( + EMPTY_METADATA, new byte[] {0b10011, (byte) 0xFF, (byte) 0x01, 0x00, 0x00}); + + assertThat(array.type()).isEqualTo(PhysicalType.ARRAY); + assertThat(array.numElements()).isEqualTo(511); + } + + @Test + public void testNegativeArraySize() { + assertThatThrownBy( + () -> + SerializedArray.from( + EMPTY_METADATA, + new byte[] {0b10011, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF})) + .isInstanceOf(NegativeArraySizeException.class) + .hasMessage("-1"); + } +} diff --git a/core/src/test/java/org/apache/iceberg/variants/TestSerializedMetadata.java b/core/src/test/java/org/apache/iceberg/variants/TestSerializedMetadata.java new file mode 100644 index 000000000000..27a4dda3dc3e --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/variants/TestSerializedMetadata.java @@ -0,0 +1,219 @@ +/* + * 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.variants; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.nio.ByteBuffer; +import java.util.Random; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.RandomUtil; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestSerializedMetadata { + private final Random random = new Random(872591); + + @Test + public void testEmptyVariantMetadata() { + SerializedMetadata metadata = SerializedMetadata.from(SerializedMetadata.EMPTY_V1_BUFFER); + + assertThat(metadata.isSorted()).isFalse(); + assertThat(metadata.dictionarySize()).isEqualTo(0); + assertThatThrownBy(() -> metadata.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); + } + + @Test + public void testHeaderSorted() { + SerializedMetadata metadata = SerializedMetadata.from(new byte[] {0b10001, 0x00}); + + assertThat(metadata.isSorted()).isTrue(); + assertThat(metadata.dictionarySize()).isEqualTo(0); + } + + @Test + public void testHeaderOffsetSize() { + // offset size is 4-byte LE = 1 + assertThat( + SerializedMetadata.from(new byte[] {(byte) 0b11010001, 0x01, 0x00, 0x00, 0x00}) + .dictionarySize()) + .isEqualTo(1); + + // offset size is 3-byte LE = 1 + assertThat( + SerializedMetadata.from(new byte[] {(byte) 0b10010001, 0x01, 0x00, 0x00}) + .dictionarySize()) + .isEqualTo(1); + + // offset size is 2-byte LE = 1 + assertThat(SerializedMetadata.from(new byte[] {(byte) 0b01010001, 0x01, 0x00}).dictionarySize()) + .isEqualTo(1); + + // offset size is 1-byte LE = 1 + assertThat(SerializedMetadata.from(new byte[] {(byte) 0b00010001, 0x01}).dictionarySize()) + .isEqualTo(1); + } + + @Test + public void testReadString() { + SerializedMetadata metadata = + SerializedMetadata.from( + new byte[] { + 0b10001, 0x05, 0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 'a', 'b', 'c', 'd', 'e' + }); + + assertThat(metadata.get(0)).isEqualTo("a"); + assertThat(metadata.get(1)).isEqualTo("b"); + assertThat(metadata.get(2)).isEqualTo("c"); + assertThat(metadata.get(3)).isEqualTo("d"); + assertThat(metadata.get(4)).isEqualTo("e"); + assertThatThrownBy(() -> metadata.get(5)).isInstanceOf(ArrayIndexOutOfBoundsException.class); + } + + @Test + public void testMultibyteString() { + SerializedMetadata metadata = + SerializedMetadata.from( + new byte[] { + 0b10001, 0x05, 0x00, 0x01, 0x02, 0x05, 0x06, 0x07, 'a', 'b', 'x', 'y', 'z', 'd', 'e' + }); + + assertThat(metadata.get(0)).isEqualTo("a"); + assertThat(metadata.get(1)).isEqualTo("b"); + assertThat(metadata.get(2)).isEqualTo("xyz"); + assertThat(metadata.get(3)).isEqualTo("d"); + assertThat(metadata.get(4)).isEqualTo("e"); + assertThatThrownBy(() -> metadata.get(5)).isInstanceOf(ArrayIndexOutOfBoundsException.class); + } + + @Test + public void testTwoByteOffsets() { + SerializedMetadata metadata = + SerializedMetadata.from( + new byte[] { + 0b1010001, 0x05, 0x00, 0x00, 0x00, 0x01, 0x00, 0x02, 0x00, 0x05, 0x00, 0x06, 0x00, + 0x07, 0x00, 'a', 'b', 'x', 'y', 'z', 'd', 'e' + }); + + assertThat(metadata.get(0)).isEqualTo("a"); + assertThat(metadata.get(1)).isEqualTo("b"); + assertThat(metadata.get(2)).isEqualTo("xyz"); + assertThat(metadata.get(3)).isEqualTo("d"); + assertThat(metadata.get(4)).isEqualTo("e"); + assertThatThrownBy(() -> metadata.get(5)).isInstanceOf(ArrayIndexOutOfBoundsException.class); + } + + @Test + public void testFindStringSorted() { + SerializedMetadata metadata = + SerializedMetadata.from( + new byte[] { + 0b10001, 0x05, 0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 'a', 'b', 'c', 'd', 'e' + }); + assertThat(metadata.id("A")).isEqualTo(-1); + assertThat(metadata.id("a")).isEqualTo(0); + assertThat(metadata.id("aa")).isEqualTo(-1); + assertThat(metadata.id("b")).isEqualTo(1); + assertThat(metadata.id("bb")).isEqualTo(-1); + assertThat(metadata.id("c")).isEqualTo(2); + assertThat(metadata.id("cc")).isEqualTo(-1); + assertThat(metadata.id("d")).isEqualTo(3); + assertThat(metadata.id("dd")).isEqualTo(-1); + assertThat(metadata.id("e")).isEqualTo(4); + assertThat(metadata.id("ee")).isEqualTo(-1); + } + + @Test + public void testFindStringUnsorted() { + SerializedMetadata metadata = + SerializedMetadata.from( + new byte[] { + 0b00001, 0x05, 0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 'e', 'd', 'c', 'b', 'a' + }); + assertThat(metadata.id("A")).isEqualTo(-1); + assertThat(metadata.id("a")).isEqualTo(4); + assertThat(metadata.id("aa")).isEqualTo(-1); + assertThat(metadata.id("b")).isEqualTo(3); + assertThat(metadata.id("bb")).isEqualTo(-1); + assertThat(metadata.id("c")).isEqualTo(2); + assertThat(metadata.id("cc")).isEqualTo(-1); + assertThat(metadata.id("d")).isEqualTo(1); + assertThat(metadata.id("dd")).isEqualTo(-1); + assertThat(metadata.id("e")).isEqualTo(0); + assertThat(metadata.id("ee")).isEqualTo(-1); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testTwoByteFieldIds(boolean sortFieldNames) { + Set keySet = Sets.newHashSet(); + String lastKey = null; + for (int i = 0; i < 10_000; i += 1) { + lastKey = RandomUtil.generateString(10, random); + keySet.add(lastKey); + } + + ByteBuffer buffer = VariantTestUtil.createMetadata(keySet, sortFieldNames); + SerializedMetadata metadata = SerializedMetadata.from(buffer); + + assertThat(metadata.dictionarySize()).isEqualTo(10_000); + assertThat(metadata.id(lastKey)).isGreaterThan(0); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testThreeByteFieldIds(boolean sortFieldNames) { + Set keySet = Sets.newHashSet(); + String lastKey = null; + for (int i = 0; i < 100_000; i += 1) { + lastKey = RandomUtil.generateString(10, random); + keySet.add(lastKey); + } + + ByteBuffer buffer = VariantTestUtil.createMetadata(keySet, sortFieldNames); + SerializedMetadata metadata = SerializedMetadata.from(buffer); + + assertThat(metadata.dictionarySize()).isEqualTo(100_000); + assertThat(metadata.id(lastKey)).isGreaterThan(0); + } + + @Test + public void testInvalidMetadataVersion() { + assertThatThrownBy(() -> SerializedMetadata.from(new byte[] {0x02, 0x00})) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported version: 2"); + } + + @Test + public void testMissingLength() { + assertThatThrownBy(() -> SerializedMetadata.from(new byte[] {0x01})) + .isInstanceOf(IndexOutOfBoundsException.class); + } + + @Test + public void testLengthTooShort() { + // missing the 4th length byte + assertThatThrownBy( + () -> SerializedMetadata.from(new byte[] {(byte) 0b11010001, 0x00, 0x00, 0x00})) + .isInstanceOf(IndexOutOfBoundsException.class); + } +} diff --git a/core/src/test/java/org/apache/iceberg/variants/TestSerializedObject.java b/core/src/test/java/org/apache/iceberg/variants/TestSerializedObject.java new file mode 100644 index 000000000000..9fe71ed0e92a --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/variants/TestSerializedObject.java @@ -0,0 +1,326 @@ +/* + * 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.variants; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.RandomUtil; +import org.apache.iceberg.variants.Variants.PhysicalType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestSerializedObject { + private static final SerializedMetadata EMPTY_METADATA = + SerializedMetadata.from(SerializedMetadata.EMPTY_V1_BUFFER); + private static final SerializedPrimitive I1 = SerializedPrimitive.from(new byte[] {0b1100, 1}); + private static final SerializedPrimitive I2 = SerializedPrimitive.from(new byte[] {0b1100, 2}); + private static final SerializedPrimitive I3 = SerializedPrimitive.from(new byte[] {0b1100, 3}); + private static final SerializedPrimitive NULL = SerializedPrimitive.from(new byte[] {0x00}); + private static final SerializedPrimitive TRUE = SerializedPrimitive.from(new byte[] {0b100}); + private static final SerializedPrimitive DATE = + SerializedPrimitive.from(new byte[] {0b101100, (byte) 0xF4, 0x43, 0x00, 0x00}); + private static final byte[] UNSORTED_VALUES = + new byte[] { + 0b10, + 0x03, // 3 item object + 0x00, + 0x01, + 0x02, // ascending key IDs (a, b, c) + 0x02, + 0x04, + 0x00, + 0x06, // values at offsets (2, 4, 0) + 0b1100, + 0x03, // c = 3 (int8) + 0b1100, + 0x01, // a = 1 (int8) + 0b1100, + 0x02 // b = 2 (int8) + }; + + private final Random random = new Random(198725); + + @Test + public void testEmptyObject() { + SerializedObject object = SerializedObject.from(EMPTY_METADATA, new byte[] {0b10, 0x00}); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(0); + } + + @Test + public void testEmptyLargeObject() { + SerializedObject object = + SerializedObject.from(EMPTY_METADATA, new byte[] {0b1000010, 0x00, 0x00, 0x00, 0x00}); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(0); + } + + @Test + public void testSimpleObject() { + Map data = ImmutableMap.of("a", I1, "b", I2, "c", I3); + ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), true /* sort names */); + ByteBuffer value = VariantTestUtil.createObject(meta, data); + + SerializedMetadata metadata = SerializedMetadata.from(meta); + SerializedObject object = SerializedObject.from(metadata, value, value.get(0)); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(3); + + assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1); + assertThat(object.get("b").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("b").asPrimitive().get()).isEqualTo((byte) 2); + assertThat(object.get("c").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("c").asPrimitive().get()).isEqualTo((byte) 3); + + assertThat(object.get("d")).isEqualTo(null); + } + + @Test + public void testUnsortedValues() { + ByteBuffer meta = + VariantTestUtil.createMetadata(Sets.newHashSet("a", "b", "c"), true /* sort names */); + + SerializedMetadata metadata = SerializedMetadata.from(meta); + SerializedObject object = SerializedObject.from(metadata, UNSORTED_VALUES); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(3); + + assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1); + assertThat(object.get("b").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("b").asPrimitive().get()).isEqualTo((byte) 2); + assertThat(object.get("c").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("c").asPrimitive().get()).isEqualTo((byte) 3); + + assertThat(object.get("d")).isEqualTo(null); + } + + @Test + public void testOutOfOrderKeys() { + Map data = ImmutableMap.of("b", I2, "a", I1, "c", I3); + ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), false /* sort names */); + ByteBuffer value = VariantTestUtil.createObject(meta, data); + + SerializedMetadata metadata = SerializedMetadata.from(meta); + SerializedObject object = SerializedObject.from(metadata, value, value.get(0)); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(3); + + assertThat(object.get("d")).isEqualTo(null); + + assertThat(object.get("c").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("c").asPrimitive().get()).isEqualTo((byte) 3); + assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1); + assertThat(object.get("b").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("b").asPrimitive().get()).isEqualTo((byte) 2); + } + + @Test + public void testMixedValueTypes() { + ByteBuffer meta = + VariantTestUtil.createMetadata( + ImmutableList.of("a", "b", "c", "d", "e", "f"), true /* sort names */); + SerializedMetadata metadata = SerializedMetadata.from(meta); + + Map inner = ImmutableMap.of("b", I2, "f", I3); + ByteBuffer innerBuffer = VariantTestUtil.createObject(meta, inner); + SerializedObject innerObject = SerializedObject.from(metadata, innerBuffer, innerBuffer.get(0)); + Map data = + ImmutableMap.of("a", I1, "b", DATE, "c", NULL, "d", TRUE, "e", innerObject); + ByteBuffer value = VariantTestUtil.createObject(meta, data); + + SerializedObject object = SerializedObject.from(metadata, value, value.get(0)); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(5); + + assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1); + assertThat(object.get("b").type()).isEqualTo(PhysicalType.DATE); + assertThat(((SerializedPrimitive) object.get("b")).get()).isEqualTo(17396); + assertThat(object.get("c").type()).isEqualTo(PhysicalType.NULL); + assertThat(((SerializedPrimitive) object.get("c")).get()).isEqualTo(null); + assertThat(object.get("d").type()).isEqualTo(PhysicalType.BOOLEAN_TRUE); + assertThat(((SerializedPrimitive) object.get("d")).get()).isEqualTo(true); + + assertThat(object.get("e").type()).isEqualTo(PhysicalType.OBJECT); + SerializedObject actualInner = (SerializedObject) object.get("e").asObject(); + assertThat(actualInner.numElements()).isEqualTo(2); + assertThat(actualInner.get("b").type()).isEqualTo(PhysicalType.INT8); + assertThat(actualInner.get("b").asPrimitive().get()).isEqualTo((byte) 2); + assertThat(actualInner.get("f").type()).isEqualTo(PhysicalType.INT8); + assertThat(actualInner.get("f").asPrimitive().get()).isEqualTo((byte) 3); + } + + @Test + public void testTwoByteOffsets() { + // a string larger than 255 bytes to push the value offset size above 1 byte + String randomString = RandomUtil.generateString(300, random); + SerializedPrimitive bigString = VariantTestUtil.createString(randomString); + + // note that order doesn't matter. fields are sorted by name + Map data = ImmutableMap.of("big", bigString, "a", I1, "b", I2, "c", I3); + ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), true /* sort names */); + ByteBuffer value = VariantTestUtil.createObject(meta, data); + + SerializedMetadata metadata = SerializedMetadata.from(meta); + SerializedObject object = SerializedObject.from(metadata, value, value.get(0)); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(4); + + assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1); + assertThat(object.get("b").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("b").asPrimitive().get()).isEqualTo((byte) 2); + assertThat(object.get("c").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("c").asPrimitive().get()).isEqualTo((byte) 3); + assertThat(object.get("big").type()).isEqualTo(PhysicalType.STRING); + assertThat(object.get("big").asPrimitive().get()).isEqualTo(randomString); + } + + @Test + public void testThreeByteOffsets() { + // a string larger than 65535 bytes to push the value offset size above 1 byte + String randomString = RandomUtil.generateString(70_000, random); + SerializedPrimitive reallyBigString = VariantTestUtil.createString(randomString); + + // note that order doesn't matter. fields are sorted by name + Map data = + ImmutableMap.of("really-big", reallyBigString, "a", I1, "b", I2, "c", I3); + ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), true /* sort names */); + ByteBuffer value = VariantTestUtil.createObject(meta, data); + + SerializedMetadata metadata = SerializedMetadata.from(meta); + SerializedObject object = SerializedObject.from(metadata, value, value.get(0)); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(4); + + assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1); + assertThat(object.get("b").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("b").asPrimitive().get()).isEqualTo((byte) 2); + assertThat(object.get("c").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("c").asPrimitive().get()).isEqualTo((byte) 3); + assertThat(object.get("really-big").type()).isEqualTo(PhysicalType.STRING); + assertThat(object.get("really-big").asPrimitive().get()).isEqualTo(randomString); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + @SuppressWarnings({"unchecked", "rawtypes"}) + public void testLargeObject(boolean sortFieldNames) { + Map> fields = Maps.newHashMap(); + for (int i = 0; i < 10_000; i += 1) { + fields.put( + RandomUtil.generateString(10, random), + Variants.of(RandomUtil.generateString(10, random))); + } + + ByteBuffer meta = VariantTestUtil.createMetadata(fields.keySet(), sortFieldNames); + ByteBuffer value = VariantTestUtil.createObject(meta, (Map) fields); + + SerializedMetadata metadata = SerializedMetadata.from(meta); + SerializedObject object = SerializedObject.from(metadata, value, value.get(0)); + + assertThat(object.type()).isEqualTo(Variants.PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(10_000); + + for (Map.Entry> entry : fields.entrySet()) { + VariantValue fieldValue = object.get(entry.getKey()); + assertThat(fieldValue.type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(fieldValue.asPrimitive().get()).isEqualTo(entry.getValue().get()); + } + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testTwoByteFieldIds(boolean sortFieldNames) { + Set keySet = Sets.newHashSet(); + for (int i = 0; i < 10_000; i += 1) { + keySet.add(RandomUtil.generateString(10, random)); + } + + Map data = ImmutableMap.of("aa", I1, "AA", I2, "ZZ", I3); + + // create metadata from the large key set and the actual keys + keySet.addAll(data.keySet()); + ByteBuffer meta = VariantTestUtil.createMetadata(keySet, sortFieldNames); + ByteBuffer value = VariantTestUtil.createObject(meta, data); + + SerializedMetadata metadata = SerializedMetadata.from(meta); + SerializedObject object = SerializedObject.from(metadata, value, value.get(0)); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(3); + + assertThat(object.get("aa").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("aa").asPrimitive().get()).isEqualTo((byte) 1); + assertThat(object.get("AA").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("AA").asPrimitive().get()).isEqualTo((byte) 2); + assertThat(object.get("ZZ").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("ZZ").asPrimitive().get()).isEqualTo((byte) 3); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testThreeByteFieldIds(boolean sortFieldNames) { + Set keySet = Sets.newHashSet(); + for (int i = 0; i < 100_000; i += 1) { + keySet.add(RandomUtil.generateString(10, random)); + } + + Map data = ImmutableMap.of("aa", I1, "AA", I2, "ZZ", I3); + + // create metadata from the large key set and the actual keys + keySet.addAll(data.keySet()); + ByteBuffer meta = VariantTestUtil.createMetadata(keySet, sortFieldNames); + ByteBuffer value = VariantTestUtil.createObject(meta, data); + + SerializedMetadata metadata = SerializedMetadata.from(meta); + SerializedObject object = SerializedObject.from(metadata, value, value.get(0)); + + assertThat(object.type()).isEqualTo(PhysicalType.OBJECT); + assertThat(object.numElements()).isEqualTo(3); + + assertThat(object.get("aa").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("aa").asPrimitive().get()).isEqualTo((byte) 1); + assertThat(object.get("AA").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("AA").asPrimitive().get()).isEqualTo((byte) 2); + assertThat(object.get("ZZ").type()).isEqualTo(PhysicalType.INT8); + assertThat(object.get("ZZ").asPrimitive().get()).isEqualTo((byte) 3); + } +} diff --git a/core/src/test/java/org/apache/iceberg/variants/TestSerializedPrimitives.java b/core/src/test/java/org/apache/iceberg/variants/TestSerializedPrimitives.java new file mode 100644 index 000000000000..b4646c0d13e9 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/variants/TestSerializedPrimitives.java @@ -0,0 +1,465 @@ +/* + * 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.variants; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.variants.Variants.PhysicalType; +import org.junit.jupiter.api.Test; + +public class TestSerializedPrimitives { + @Test + public void testNull() { + VariantPrimitive value = SerializedPrimitive.from(new byte[] {primitiveHeader(0)}); + + assertThat(value.type()).isEqualTo(PhysicalType.NULL); + assertThat(value.get()).isEqualTo(null); + } + + @Test + public void testTrue() { + VariantPrimitive value = SerializedPrimitive.from(new byte[] {primitiveHeader(1)}); + + assertThat(value.type()).isEqualTo(PhysicalType.BOOLEAN_TRUE); + assertThat(value.get()).isEqualTo(true); + } + + @Test + public void testFalse() { + VariantPrimitive value = SerializedPrimitive.from(new byte[] {primitiveHeader(2)}); + + assertThat(value.type()).isEqualTo(PhysicalType.BOOLEAN_FALSE); + assertThat(value.get()).isEqualTo(false); + } + + @Test + public void testInt8() { + VariantPrimitive value = SerializedPrimitive.from(new byte[] {primitiveHeader(3), 34}); + + assertThat(value.type()).isEqualTo(PhysicalType.INT8); + assertThat(value.get()).isEqualTo((byte) 34); + } + + @Test + public void testNegativeInt8() { + VariantPrimitive value = + SerializedPrimitive.from(new byte[] {primitiveHeader(3), (byte) 0xFF}); + + assertThat(value.type()).isEqualTo(PhysicalType.INT8); + assertThat(value.get()).isEqualTo((byte) -1); + } + + @Test + public void testInt16() { + VariantPrimitive value = + SerializedPrimitive.from(new byte[] {primitiveHeader(4), (byte) 0xD2, 0x04}); + + assertThat(value.type()).isEqualTo(PhysicalType.INT16); + assertThat(value.get()).isEqualTo((short) 1234); + } + + @Test + public void testNegativeInt16() { + VariantPrimitive value = + SerializedPrimitive.from(new byte[] {primitiveHeader(4), (byte) 0xFF, (byte) 0xFF}); + + assertThat(value.type()).isEqualTo(PhysicalType.INT16); + assertThat(value.get()).isEqualTo((short) -1); + } + + @Test + public void testInt32() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] {primitiveHeader(5), (byte) 0xD2, 0x02, (byte) 0x96, 0x49}); + + assertThat(value.type()).isEqualTo(PhysicalType.INT32); + assertThat(value.get()).isEqualTo(1234567890); + } + + @Test + public void testNegativeInt32() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] {primitiveHeader(5), (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF}); + + assertThat(value.type()).isEqualTo(PhysicalType.INT32); + assertThat(value.get()).isEqualTo(-1); + } + + @Test + public void testInt64() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(6), + (byte) 0xB1, + 0x1C, + 0x6C, + (byte) 0xB1, + (byte) 0xF4, + 0x10, + 0x22, + 0x11 + }); + + assertThat(value.type()).isEqualTo(PhysicalType.INT64); + assertThat(value.get()).isEqualTo(1234567890987654321L); + } + + @Test + public void testNegativeInt64() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(6), + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF + }); + + assertThat(value.type()).isEqualTo(PhysicalType.INT64); + assertThat(value.get()).isEqualTo(-1L); + } + + @Test + public void testDouble() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(7), + (byte) 0xB1, + 0x1C, + 0x6C, + (byte) 0xB1, + (byte) 0xF4, + 0x10, + 0x22, + 0x11 + }); + + assertThat(value.type()).isEqualTo(PhysicalType.DOUBLE); + assertThat(value.get()).isEqualTo(Double.longBitsToDouble(1234567890987654321L)); + } + + @Test + public void testNegativeDouble() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] {primitiveHeader(7), 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, (byte) 0x80}); + + assertThat(value.type()).isEqualTo(PhysicalType.DOUBLE); + assertThat(value.get()).isEqualTo(-0.0D); + } + + @Test + public void testDecimal4() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] {primitiveHeader(8), 0x04, (byte) 0xD2, 0x02, (byte) 0x96, 0x49}); + + assertThat(value.type()).isEqualTo(PhysicalType.DECIMAL4); + assertThat(value.get()).isEqualTo(new BigDecimal("123456.7890")); + } + + @Test + public void testNegativeDecimal4() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(8), 0x04, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF + }); + + assertThat(value.type()).isEqualTo(PhysicalType.DECIMAL4); + assertThat(value.get()).isEqualTo(new BigDecimal("-0.0001")); + } + + @Test + public void testDecimal8() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(9), + 0x09, // scale=9 + (byte) 0xB1, + 0x1C, + 0x6C, + (byte) 0xB1, + (byte) 0xF4, + 0x10, + 0x22, + 0x11 + }); + + assertThat(value.type()).isEqualTo(PhysicalType.DECIMAL8); + assertThat(value.get()).isEqualTo(new BigDecimal("1234567890.987654321")); + } + + @Test + public void testNegativeDecimal8() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(9), + 0x09, // scale=9 + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF + }); + + assertThat(value.type()).isEqualTo(PhysicalType.DECIMAL8); + assertThat(value.get()).isEqualTo(new BigDecimal("-0.000000001")); + } + + @Test + public void testDecimal16() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(10), + 0x09, // scale=9 + 0x15, + 0x71, + 0x34, + (byte) 0xB0, + (byte) 0xB8, + (byte) 0x87, + 0x10, + (byte) 0x89, + 0x00, + 0x00, + 0x00, + 0x00, + 0x00, + 0x00, + 0x00, + 0x00 + }); + + assertThat(value.type()).isEqualTo(PhysicalType.DECIMAL16); + assertThat(value.get()).isEqualTo(new BigDecimal("9876543210.123456789")); + } + + @Test + public void testNegativeDecimal16() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(10), + 0x09, // scale=9 + (byte) 0xEB, + (byte) 0x8E, + (byte) 0xCB, + 0x4F, + 0x47, + 0x78, + (byte) 0xEF, + 0x76, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + }); + + assertThat(value.type()).isEqualTo(PhysicalType.DECIMAL16); + assertThat(value.get()).isEqualTo(new BigDecimal("-9876543210.123456789")); + } + + @Test + public void testDate() { + VariantPrimitive value = + SerializedPrimitive.from(new byte[] {primitiveHeader(11), (byte) 0xF4, 0x43, 0x00, 0x00}); + + assertThat(value.type()).isEqualTo(PhysicalType.DATE); + assertThat(DateTimeUtil.daysToIsoDate((int) value.get())).isEqualTo("2017-08-18"); + } + + @Test + public void testNegativeDate() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] {primitiveHeader(11), (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF}); + + assertThat(value.type()).isEqualTo(PhysicalType.DATE); + assertThat(DateTimeUtil.daysToIsoDate((int) value.get())).isEqualTo("1969-12-31"); + } + + @Test + public void testTimestamptz() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(12), + 0x18, + (byte) 0xD3, + (byte) 0xB1, + (byte) 0xD6, + 0x07, + 0x57, + 0x05, + 0x00 + }); + + assertThat(value.type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(DateTimeUtil.microsToIsoTimestamptz((long) value.get())) + .isEqualTo("2017-08-18T14:21:01.919+00:00"); + } + + @Test + public void testNegativeTimestamptz() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(12), + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF + }); + + assertThat(value.type()).isEqualTo(PhysicalType.TIMESTAMPTZ); + assertThat(DateTimeUtil.microsToIsoTimestamptz((long) value.get())) + .isEqualTo("1969-12-31T23:59:59.999999+00:00"); + } + + @Test + public void testTimestampntz() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(13), + 0x18, + (byte) 0xD3, + (byte) 0xB1, + (byte) 0xD6, + 0x07, + 0x57, + 0x05, + 0x00 + }); + + assertThat(value.type()).isEqualTo(PhysicalType.TIMESTAMPNTZ); + assertThat(DateTimeUtil.microsToIsoTimestamp((long) value.get())) + .isEqualTo("2017-08-18T14:21:01.919"); + } + + @Test + public void testNegativeTimestampntz() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(13), + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF, + (byte) 0xFF + }); + + assertThat(value.type()).isEqualTo(PhysicalType.TIMESTAMPNTZ); + assertThat(DateTimeUtil.microsToIsoTimestamp((long) value.get())) + .isEqualTo("1969-12-31T23:59:59.999999"); + } + + @Test + public void testFloat() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] {primitiveHeader(14), (byte) 0xD2, 0x02, (byte) 0x96, 0x49}); + + assertThat(value.type()).isEqualTo(PhysicalType.FLOAT); + assertThat(value.get()).isEqualTo(Float.intBitsToFloat(1234567890)); + } + + @Test + public void testNegativeFloat() { + VariantPrimitive value = + SerializedPrimitive.from(new byte[] {primitiveHeader(14), 0x00, 0x00, 0x00, (byte) 0x80}); + + assertThat(value.type()).isEqualTo(PhysicalType.FLOAT); + assertThat(value.get()).isEqualTo(-0.0F); + } + + @Test + public void testBinary() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] {primitiveHeader(15), 0x05, 0x00, 0x00, 0x00, 'a', 'b', 'c', 'd', 'e'}); + + assertThat(value.type()).isEqualTo(PhysicalType.BINARY); + assertThat(value.get()).isEqualTo(ByteBuffer.wrap(new byte[] {'a', 'b', 'c', 'd', 'e'})); + } + + @Test + public void testString() { + VariantPrimitive value = + SerializedPrimitive.from( + new byte[] { + primitiveHeader(16), 0x07, 0x00, 0x00, 0x00, 'i', 'c', 'e', 'b', 'e', 'r', 'g' + }); + + assertThat(value.type()).isEqualTo(PhysicalType.STRING); + assertThat(value.get()).isEqualTo("iceberg"); + } + + @Test + public void testShortString() { + VariantPrimitive value = + SerializedShortString.from(new byte[] {0b11101, 'i', 'c', 'e', 'b', 'e', 'r', 'g'}); + + assertThat(value.type()).isEqualTo(PhysicalType.STRING); + assertThat(value.get()).isEqualTo("iceberg"); + } + + @Test + public void testUnsupportedType() { + assertThatThrownBy(() -> SerializedPrimitive.from(new byte[] {primitiveHeader(17)})) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Unknown primitive physical type: 17"); + } + + private static byte primitiveHeader(int primitiveType) { + return (byte) (primitiveType << 2); + } +} diff --git a/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java b/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java new file mode 100644 index 000000000000..0a2c2b26b004 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java @@ -0,0 +1,448 @@ +/* + * 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.variants; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.RandomUtil; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestShreddedObject { + private static final Map FIELDS = + ImmutableMap.of( + "a", + Variants.of(34), + "b", + Variants.of("iceberg"), + "c", + Variants.of(new BigDecimal("12.21"))); + + private final Random random = new Random(871925); + + @Test + public void testShreddedFields() { + ShreddedObject object = createShreddedObject(FIELDS).second(); + + assertThat(object.get("a")).isInstanceOf(VariantPrimitive.class); + assertThat(object.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(object.get("b")).isInstanceOf(VariantPrimitive.class); + assertThat(object.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(object.get("c")).isInstanceOf(VariantPrimitive.class); + assertThat(object.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + } + + @Test + public void testShreddedSerializationMinimalBuffer() { + Pair pair = createShreddedObject(FIELDS); + SerializedMetadata metadata = pair.first(); + ShreddedObject object = pair.second(); + + VariantValue value = roundTripMinimalBuffer(object, metadata); + + assertThat(value).isInstanceOf(SerializedObject.class); + SerializedObject actual = (SerializedObject) value; + + assertThat(actual.numElements()).isEqualTo(3); + assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(actual.get("c")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + } + + @Test + public void testShreddedSerializationLargeBuffer() { + Pair pair = createShreddedObject(FIELDS); + SerializedMetadata metadata = pair.first(); + ShreddedObject object = pair.second(); + + VariantValue value = roundTripLargeBuffer(object, metadata); + + assertThat(value).isInstanceOf(SerializedObject.class); + SerializedObject actual = (SerializedObject) value; + + assertThat(actual.numElements()).isEqualTo(3); + assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(actual.get("c")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + } + + @Test + public void testUnshreddedObjectSerializationMinimalBuffer() { + Pair pair = createUnshreddedObject(FIELDS); + SerializedMetadata metadata = pair.first(); + ShreddedObject object = pair.second(); + + VariantValue value = roundTripMinimalBuffer(object, metadata); + + assertThat(value).isInstanceOf(SerializedObject.class); + SerializedObject actual = (SerializedObject) value; + + assertThat(actual.numElements()).isEqualTo(3); + assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(actual.get("c")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + } + + @Test + public void testUnshreddedObjectSerializationLargeBuffer() { + Pair pair = createUnshreddedObject(FIELDS); + SerializedMetadata metadata = pair.first(); + ShreddedObject object = pair.second(); + + VariantValue value = roundTripLargeBuffer(object, metadata); + + assertThat(value).isInstanceOf(SerializedObject.class); + SerializedObject actual = (SerializedObject) value; + + assertThat(actual.numElements()).isEqualTo(3); + assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(actual.get("c")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + } + + @Test + public void testPartiallyShreddedObjectReplacement() { + ShreddedObject partial = createUnshreddedObject(FIELDS).second(); + + // replace field c with a new value + partial.put("c", Variants.ofIsoDate("2024-10-12")); + + assertThat(partial.get("a")).isInstanceOf(VariantPrimitive.class); + assertThat(partial.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(partial.get("b")).isInstanceOf(VariantPrimitive.class); + assertThat(partial.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(partial.get("c")).isInstanceOf(VariantPrimitive.class); + assertThat(partial.get("c").type()).isEqualTo(Variants.PhysicalType.DATE); + assertThat(partial.get("c").asPrimitive().get()) + .isEqualTo(DateTimeUtil.isoDateToDays("2024-10-12")); + } + + @Test + public void testPartiallyShreddedObjectGetMissingField() { + ShreddedObject partial = createUnshreddedObject(FIELDS).second(); + + // missing fields are returned as null + assertThat(partial.get("d")).isNull(); + } + + @Test + public void testPartiallyShreddedObjectPutMissingFieldFailure() { + ShreddedObject partial = createUnshreddedObject(FIELDS).second(); + + // d is not defined in the variant metadata and will fail + assertThatThrownBy(() -> partial.put("d", Variants.ofIsoDate("2024-10-12"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find field name in metadata: d"); + } + + @Test + public void testPartiallyShreddedObjectSerializationMinimalBuffer() { + Pair pair = createUnshreddedObject(FIELDS); + SerializedMetadata metadata = pair.first(); + ShreddedObject partial = pair.second(); + + // replace field c with a new value + partial.put("c", Variants.ofIsoDate("2024-10-12")); + + VariantValue value = roundTripMinimalBuffer(partial, metadata); + + assertThat(value).isInstanceOf(SerializedObject.class); + SerializedObject actual = (SerializedObject) value; + + assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(actual.get("c")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("c").type()).isEqualTo(Variants.PhysicalType.DATE); + assertThat(actual.get("c").asPrimitive().get()) + .isEqualTo(DateTimeUtil.isoDateToDays("2024-10-12")); + } + + @Test + public void testPartiallyShreddedObjectSerializationLargeBuffer() { + Pair pair = createUnshreddedObject(FIELDS); + SerializedMetadata metadata = pair.first(); + ShreddedObject partial = pair.second(); + + // replace field c with a new value + partial.put("c", Variants.ofIsoDate("2024-10-12")); + + VariantValue value = roundTripLargeBuffer(partial, metadata); + + assertThat(value).isInstanceOf(SerializedObject.class); + SerializedObject actual = (SerializedObject) value; + + assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(actual.get("c")).isInstanceOf(VariantPrimitive.class); + assertThat(actual.get("c").type()).isEqualTo(Variants.PhysicalType.DATE); + assertThat(actual.get("c").asPrimitive().get()) + .isEqualTo(DateTimeUtil.isoDateToDays("2024-10-12")); + } + + @Test + public void testTwoByteOffsets() { + // a string larger than 255 bytes to push the value offset size above 1 byte + String randomString = RandomUtil.generateString(300, random); + SerializedPrimitive bigString = VariantTestUtil.createString(randomString); + + Map data = Maps.newHashMap(); + data.putAll(FIELDS); + data.put("big", bigString); + + Pair pair = createShreddedObject(data); + VariantValue value = roundTripLargeBuffer(pair.second(), pair.first()); + + assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT); + SerializedObject object = (SerializedObject) value; + assertThat(object.numElements()).isEqualTo(4); + + assertThat(object.get("a").type()).isEqualTo(Variants.PhysicalType.INT32); + assertThat(object.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(object.get("b").type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(object.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(object.get("c").type()).isEqualTo(Variants.PhysicalType.DECIMAL4); + assertThat(object.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + assertThat(object.get("big").type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(object.get("big").asPrimitive().get()).isEqualTo(randomString); + } + + @Test + public void testThreeByteOffsets() { + // a string larger than 65535 bytes to push the value offset size above 2 bytes + String randomString = RandomUtil.generateString(70_000, random); + SerializedPrimitive reallyBigString = VariantTestUtil.createString(randomString); + + Map data = Maps.newHashMap(); + data.putAll(FIELDS); + data.put("really-big", reallyBigString); + + Pair pair = createShreddedObject(data); + VariantValue value = roundTripLargeBuffer(pair.second(), pair.first()); + + assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT); + SerializedObject object = (SerializedObject) value; + assertThat(object.numElements()).isEqualTo(4); + + assertThat(object.get("a").type()).isEqualTo(Variants.PhysicalType.INT32); + assertThat(object.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(object.get("b").type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(object.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(object.get("c").type()).isEqualTo(Variants.PhysicalType.DECIMAL4); + assertThat(object.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + assertThat(object.get("really-big").type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(object.get("really-big").asPrimitive().get()).isEqualTo(randomString); + } + + @Test + public void testFourByteOffsets() { + // a string larger than 16777215 bytes to push the value offset size above 3 bytes + String randomString = RandomUtil.generateString(16_777_300, random); + SerializedPrimitive reallyBigString = VariantTestUtil.createString(randomString); + + Map data = Maps.newHashMap(); + data.putAll(FIELDS); + data.put("really-big", reallyBigString); + + Pair pair = createShreddedObject(data); + VariantValue value = roundTripLargeBuffer(pair.second(), pair.first()); + + assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT); + SerializedObject object = (SerializedObject) value; + assertThat(object.numElements()).isEqualTo(4); + + assertThat(object.get("a").type()).isEqualTo(Variants.PhysicalType.INT32); + assertThat(object.get("a").asPrimitive().get()).isEqualTo(34); + assertThat(object.get("b").type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(object.get("b").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(object.get("c").type()).isEqualTo(Variants.PhysicalType.DECIMAL4); + assertThat(object.get("c").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + assertThat(object.get("really-big").type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(object.get("really-big").asPrimitive().get()).isEqualTo(randomString); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + @SuppressWarnings({"unchecked", "rawtypes"}) + public void testLargeObject(boolean sortFieldNames) { + Map> fields = Maps.newHashMap(); + for (int i = 0; i < 10_000; i += 1) { + fields.put( + RandomUtil.generateString(10, random), + Variants.of(RandomUtil.generateString(10, random))); + } + + SerializedMetadata metadata = + SerializedMetadata.from(VariantTestUtil.createMetadata(fields.keySet(), sortFieldNames)); + + ShreddedObject shredded = createShreddedObject(metadata, (Map) fields); + VariantValue value = roundTripLargeBuffer(shredded, metadata); + + assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT); + SerializedObject object = (SerializedObject) value; + assertThat(object.numElements()).isEqualTo(10_000); + + for (Map.Entry> entry : fields.entrySet()) { + VariantValue fieldValue = object.get(entry.getKey()); + assertThat(fieldValue.type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(fieldValue.asPrimitive().get()).isEqualTo(entry.getValue().get()); + } + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testTwoByteFieldIds(boolean sortFieldNames) { + Set keySet = Sets.newHashSet(); + for (int i = 0; i < 10_000; i += 1) { + keySet.add(RandomUtil.generateString(10, random)); + } + + Map data = + ImmutableMap.of("aa", FIELDS.get("a"), "AA", FIELDS.get("b"), "ZZ", FIELDS.get("c")); + + // create metadata from the large key set and the actual keys + keySet.addAll(data.keySet()); + SerializedMetadata metadata = + SerializedMetadata.from(VariantTestUtil.createMetadata(keySet, sortFieldNames)); + + ShreddedObject shredded = createShreddedObject(metadata, data); + VariantValue value = roundTripLargeBuffer(shredded, metadata); + + assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT); + SerializedObject object = (SerializedObject) value; + assertThat(object.numElements()).isEqualTo(3); + + assertThat(object.get("aa").type()).isEqualTo(Variants.PhysicalType.INT32); + assertThat(object.get("aa").asPrimitive().get()).isEqualTo(34); + assertThat(object.get("AA").type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(object.get("AA").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(object.get("ZZ").type()).isEqualTo(Variants.PhysicalType.DECIMAL4); + assertThat(object.get("ZZ").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testThreeByteFieldIds(boolean sortFieldNames) { + Set keySet = Sets.newHashSet(); + for (int i = 0; i < 100_000; i += 1) { + keySet.add(RandomUtil.generateString(10, random)); + } + + Map data = + ImmutableMap.of("aa", FIELDS.get("a"), "AA", FIELDS.get("b"), "ZZ", FIELDS.get("c")); + + // create metadata from the large key set and the actual keys + keySet.addAll(data.keySet()); + SerializedMetadata metadata = + SerializedMetadata.from(VariantTestUtil.createMetadata(keySet, sortFieldNames)); + + ShreddedObject shredded = createShreddedObject(metadata, data); + VariantValue value = roundTripLargeBuffer(shredded, metadata); + + assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT); + SerializedObject object = (SerializedObject) value; + assertThat(object.numElements()).isEqualTo(3); + + assertThat(object.get("aa").type()).isEqualTo(Variants.PhysicalType.INT32); + assertThat(object.get("aa").asPrimitive().get()).isEqualTo(34); + assertThat(object.get("AA").type()).isEqualTo(Variants.PhysicalType.STRING); + assertThat(object.get("AA").asPrimitive().get()).isEqualTo("iceberg"); + assertThat(object.get("ZZ").type()).isEqualTo(Variants.PhysicalType.DECIMAL4); + assertThat(object.get("ZZ").asPrimitive().get()).isEqualTo(new BigDecimal("12.21")); + } + + private static VariantValue roundTripMinimalBuffer( + ShreddedObject object, SerializedMetadata metadata) { + ByteBuffer serialized = + ByteBuffer.allocate(object.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN); + object.writeTo(serialized, 0); + + return Variants.from(metadata, serialized); + } + + private static VariantValue roundTripLargeBuffer( + ShreddedObject object, SerializedMetadata metadata) { + ByteBuffer serialized = + ByteBuffer.allocate(1000 + object.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN); + object.writeTo(serialized, 300); + + ByteBuffer slice = serialized.duplicate().order(ByteOrder.LITTLE_ENDIAN); + slice.position(300); + slice.limit(300 + object.sizeInBytes()); + + return Variants.from(metadata, slice); + } + + private static ShreddedObject createShreddedObject( + SerializedMetadata metadata, Map fields) { + ShreddedObject object = new ShreddedObject(metadata); + for (Map.Entry field : fields.entrySet()) { + object.put(field.getKey(), field.getValue()); + } + + return object; + } + + private static Pair createShreddedObject( + Map fields) { + ByteBuffer metadataBuffer = VariantTestUtil.createMetadata(fields.keySet(), false); + SerializedMetadata metadata = SerializedMetadata.from(metadataBuffer); + return Pair.of(metadata, createShreddedObject(metadata, fields)); + } + + private static Pair createUnshreddedObject( + Map fields) { + SerializedObject serialized = createSerializedObject(fields); + return Pair.of(serialized.metadata(), new ShreddedObject(serialized)); + } + + private static SerializedObject createSerializedObject(Map fields) { + ByteBuffer metadataBuffer = VariantTestUtil.createMetadata(fields.keySet(), false); + return (SerializedObject) + Variants.from(metadataBuffer, VariantTestUtil.createObject(metadataBuffer, fields)); + } +} diff --git a/core/src/test/java/org/apache/iceberg/variants/TestVariantUtil.java b/core/src/test/java/org/apache/iceberg/variants/TestVariantUtil.java new file mode 100644 index 000000000000..9d9536fbf0d6 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/variants/TestVariantUtil.java @@ -0,0 +1,44 @@ +/* + * 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.variants; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import org.junit.jupiter.api.Test; + +public class TestVariantUtil { + @Test + public void testReadByteUnsigned() { + ByteBuffer buffer = ByteBuffer.wrap(new byte[] {(byte) 0xFF}); + assertThat(VariantUtil.readByte(buffer, 0)).isEqualTo(255); + } + + @Test + public void testRead2ByteUnsigned() { + ByteBuffer buffer = ByteBuffer.wrap(new byte[] {(byte) 0xFF, (byte) 0xFF}); + assertThat(VariantUtil.readLittleEndianUnsigned(buffer, 0, 2)).isEqualTo(65535); + } + + @Test + public void testRead3ByteUnsigned() { + ByteBuffer buffer = ByteBuffer.wrap(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF}); + assertThat(VariantUtil.readLittleEndianUnsigned(buffer, 0, 3)).isEqualTo(16777215); + } +} diff --git a/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java b/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java new file mode 100644 index 000000000000..b6caec63758a --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java @@ -0,0 +1,214 @@ +/* + * 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.variants; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class VariantTestUtil { + private VariantTestUtil() {} + + private static byte primitiveHeader(int primitiveType) { + return (byte) (primitiveType << 2); + } + + private static byte metadataHeader(boolean isSorted, int offsetSize) { + return (byte) (((offsetSize - 1) << 6) | (isSorted ? 0b10000 : 0) | 0b0001); + } + + /** A hacky absolute put for ByteBuffer */ + private static int writeBufferAbsolute(ByteBuffer buffer, int offset, ByteBuffer toCopy) { + int originalPosition = buffer.position(); + buffer.position(offset); + ByteBuffer copy = toCopy.duplicate(); + buffer.put(copy); // duplicate so toCopy is not modified + buffer.position(originalPosition); + Preconditions.checkArgument(copy.remaining() <= 0, "Not fully written"); + return toCopy.remaining(); + } + + /** Creates a random string primitive of the given length for forcing large offset sizes */ + static SerializedPrimitive createString(String string) { + byte[] utf8 = string.getBytes(StandardCharsets.UTF_8); + ByteBuffer buffer = ByteBuffer.allocate(5 + utf8.length).order(ByteOrder.LITTLE_ENDIAN); + buffer.put(0, primitiveHeader(16)); + buffer.putInt(1, utf8.length); + writeBufferAbsolute(buffer, 5, ByteBuffer.wrap(utf8)); + return SerializedPrimitive.from(buffer, buffer.get(0)); + } + + static ByteBuffer createMetadata(Collection fieldNames, boolean sortNames) { + if (fieldNames.isEmpty()) { + return SerializedMetadata.EMPTY_V1_BUFFER; + } + + int numElements = fieldNames.size(); + Stream names = sortNames ? fieldNames.stream().sorted() : fieldNames.stream(); + ByteBuffer[] nameBuffers = + names + .map(str -> ByteBuffer.wrap(str.getBytes(StandardCharsets.UTF_8))) + .toArray(ByteBuffer[]::new); + + int dataSize = 0; + for (ByteBuffer nameBuffer : nameBuffers) { + dataSize += nameBuffer.remaining(); + } + + int offsetSize = VariantUtil.sizeOf(dataSize); + int offsetListOffset = 1 /* header size */ + offsetSize /* dictionary size */; + int dataOffset = offsetListOffset + ((1 + numElements) * offsetSize); + int totalSize = dataOffset + dataSize; + + byte header = metadataHeader(sortNames, offsetSize); + ByteBuffer buffer = ByteBuffer.allocate(totalSize).order(ByteOrder.LITTLE_ENDIAN); + + buffer.put(0, header); + VariantUtil.writeLittleEndianUnsigned(buffer, numElements, 1, offsetSize); + + // write offsets and strings + int nextOffset = 0; + int index = 0; + for (ByteBuffer nameBuffer : nameBuffers) { + // write the offset and the string + VariantUtil.writeLittleEndianUnsigned( + buffer, nextOffset, offsetListOffset + (index * offsetSize), offsetSize); + int nameSize = writeBufferAbsolute(buffer, dataOffset + nextOffset, nameBuffer); + // update the offset and index + nextOffset += nameSize; + index += 1; + } + + // write the final size of the data section + VariantUtil.writeLittleEndianUnsigned( + buffer, nextOffset, offsetListOffset + (index * offsetSize), offsetSize); + + return buffer; + } + + static ByteBuffer createObject(ByteBuffer metadataBuffer, Map data) { + // create the metadata to look up field names + SerializedMetadata metadata = SerializedMetadata.from(metadataBuffer); + + int numElements = data.size(); + boolean isLarge = numElements > 0xFF; + + int dataSize = 0; + for (Map.Entry field : data.entrySet()) { + dataSize += field.getValue().sizeInBytes(); + } + + // field ID size is the size needed to store the largest field ID in the data + int fieldIdSize = VariantUtil.sizeOf(metadata.dictionarySize()); + int fieldIdListOffset = 1 /* header size */ + (isLarge ? 4 : 1) /* num elements size */; + + // offset size is the size needed to store the length of the data section + int offsetSize = VariantUtil.sizeOf(dataSize); + int offsetListOffset = fieldIdListOffset + (numElements * fieldIdSize); + int dataOffset = offsetListOffset + ((1 + numElements) * offsetSize); + int totalSize = dataOffset + dataSize; + + byte header = VariantUtil.objectHeader(isLarge, fieldIdSize, offsetSize); + ByteBuffer buffer = ByteBuffer.allocate(totalSize).order(ByteOrder.LITTLE_ENDIAN); + + buffer.put(0, header); + if (isLarge) { + buffer.putInt(1, numElements); + } else { + buffer.put(1, (byte) (numElements & 0xFF)); + } + + // write field IDs, values, and offsets + int nextOffset = 0; + int index = 0; + List sortedFieldNames = data.keySet().stream().sorted().collect(Collectors.toList()); + for (String fieldName : sortedFieldNames) { + int id = metadata.id(fieldName); + VariantUtil.writeLittleEndianUnsigned( + buffer, id, fieldIdListOffset + (index * fieldIdSize), fieldIdSize); + VariantUtil.writeLittleEndianUnsigned( + buffer, nextOffset, offsetListOffset + (index * offsetSize), offsetSize); + int valueSize = data.get(fieldName).writeTo(buffer, dataOffset + nextOffset); + + // update next offset and index + nextOffset += valueSize; + index += 1; + } + + // write the final size of the data section + VariantUtil.writeLittleEndianUnsigned( + buffer, nextOffset, offsetListOffset + (index * offsetSize), offsetSize); + + return buffer; + } + + static ByteBuffer createArray(Variants.Serialized... values) { + int numElements = values.length; + boolean isLarge = numElements > 0xFF; + + int dataSize = 0; + for (Variants.Serialized value : values) { + // TODO: produce size for every variant without serializing + dataSize += value.buffer().remaining(); + } + + // offset size is the size needed to store the length of the data section + int offsetSize = VariantUtil.sizeOf(dataSize); + int offsetListOffset = 1 /* header size */ + (isLarge ? 4 : 1) /* num elements size */; + int dataOffset = offsetListOffset + ((1 + numElements) * offsetSize) /* offset list size */; + int totalSize = dataOffset + dataSize; + + byte header = VariantUtil.arrayHeader(isLarge, offsetSize); + ByteBuffer buffer = ByteBuffer.allocate(totalSize).order(ByteOrder.LITTLE_ENDIAN); + + buffer.put(0, header); + if (isLarge) { + buffer.putInt(1, numElements); + } else { + buffer.put(1, (byte) (numElements & 0xFF)); + } + + // write values and offsets + int nextOffset = 0; // the first offset is always 0 + int index = 0; + for (Variants.Serialized value : values) { + // write the offset and value + VariantUtil.writeLittleEndianUnsigned( + buffer, nextOffset, offsetListOffset + (index * offsetSize), offsetSize); + // in a real implementation, the buffer should be passed to serialize + ByteBuffer valueBuffer = value.buffer(); + int valueSize = writeBufferAbsolute(buffer, dataOffset + nextOffset, valueBuffer); + // update next offset and index + nextOffset += valueSize; + index += 1; + } + + // write the final size of the data section + VariantUtil.writeLittleEndianUnsigned( + buffer, nextOffset, offsetListOffset + (index * offsetSize), offsetSize); + + return buffer; + } +} From cd187c5718ba1eb0c8853d595eed82fa7230bc3d Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Fri, 20 Dec 2024 16:31:48 -0800 Subject: [PATCH 281/313] Spark: Test reading default values in Spark (#11832) --- .../org/apache/iceberg/types/ReassignIds.java | 6 +- .../java/org/apache/iceberg/types/Types.java | 8 +- .../apache/iceberg/types/TestTypeUtil.java | 20 +- .../java/org/apache/iceberg/SchemaParser.java | 45 +- .../org/apache/iceberg/TestSchemaParser.java | 126 ++++++ .../iceberg/spark/source/BaseRowReader.java | 2 +- .../iceberg/spark/data/AvroDataTest.java | 53 +-- .../spark/data/ParameterizedAvroDataTest.java | 284 ------------ .../iceberg/spark/data/TestHelpers.java | 35 +- .../spark/source/DataFrameWriteTestBase.java | 140 ++++++ .../iceberg/spark/source/ScanTestBase.java | 126 ++++++ .../spark/source/TestAvroDataFrameWrite.java | 33 ++ .../iceberg/spark/source/TestAvroScan.java | 64 +-- .../spark/source/TestDataFrameWrites.java | 412 ------------------ .../spark/source/TestORCDataFrameWrite.java | 33 ++ .../source/TestParquetDataFrameWrite.java | 33 ++ .../iceberg/spark/source/TestParquetScan.java | 136 +----- .../source/TestParquetVectorizedScan.java | 26 ++ 18 files changed, 644 insertions(+), 938 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/TestSchemaParser.java delete mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/ParameterizedAvroDataTest.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java delete mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java diff --git a/api/src/main/java/org/apache/iceberg/types/ReassignIds.java b/api/src/main/java/org/apache/iceberg/types/ReassignIds.java index d31fea98e53f..565ceee2a901 100644 --- a/api/src/main/java/org/apache/iceberg/types/ReassignIds.java +++ b/api/src/main/java/org/apache/iceberg/types/ReassignIds.java @@ -79,11 +79,7 @@ public Type struct(Types.StructType struct, Iterable fieldTypes) { for (int i = 0; i < length; i += 1) { Types.NestedField field = fields.get(i); int fieldId = id(sourceStruct, field.name()); - if (field.isRequired()) { - newFields.add(Types.NestedField.required(fieldId, field.name(), types.get(i), field.doc())); - } else { - newFields.add(Types.NestedField.optional(fieldId, field.name(), types.get(i), field.doc())); - } + newFields.add(Types.NestedField.from(field).withId(fieldId).ofType(types.get(i)).build()); } return Types.StructType.of(newFields); diff --git a/api/src/main/java/org/apache/iceberg/types/Types.java b/api/src/main/java/org/apache/iceberg/types/Types.java index 927b3a5065ad..3c03a3defb42 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -711,8 +711,14 @@ public boolean equals(Object o) { return false; } else if (!Objects.equals(doc, that.doc)) { return false; + } else if (!type.equals(that.type)) { + return false; + } else if (!Objects.equals(initialDefault, that.initialDefault)) { + return false; + } else if (!Objects.equals(writeDefault, that.writeDefault)) { + return false; } - return type.equals(that.type); + return true; } @Override diff --git a/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java b/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java index e8db0937eb73..36384d232af3 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java @@ -595,7 +595,12 @@ public void testReassignOrRefreshIds() { new Schema( Lists.newArrayList( required(10, "a", Types.IntegerType.get()), - required(11, "c", Types.IntegerType.get()), + Types.NestedField.required("c") + .withId(11) + .ofType(Types.IntegerType.get()) + .withInitialDefault(23) + .withWriteDefault(34) + .build(), required(12, "B", Types.IntegerType.get())), Sets.newHashSet(10)); Schema sourceSchema = @@ -603,13 +608,20 @@ public void testReassignOrRefreshIds() { Lists.newArrayList( required(1, "a", Types.IntegerType.get()), required(15, "B", Types.IntegerType.get()))); - final Schema actualSchema = TypeUtil.reassignOrRefreshIds(schema, sourceSchema); - final Schema expectedSchema = + + Schema actualSchema = TypeUtil.reassignOrRefreshIds(schema, sourceSchema); + Schema expectedSchema = new Schema( Lists.newArrayList( required(1, "a", Types.IntegerType.get()), - required(16, "c", Types.IntegerType.get()), + Types.NestedField.required("c") + .withId(16) + .ofType(Types.IntegerType.get()) + .withInitialDefault(23) + .withWriteDefault(34) + .build(), required(15, "B", Types.IntegerType.get()))); + assertThat(actualSchema.asStruct()).isEqualTo(expectedSchema.asStruct()); } diff --git a/core/src/main/java/org/apache/iceberg/SchemaParser.java b/core/src/main/java/org/apache/iceberg/SchemaParser.java index a4333af1be19..27e6ed048712 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaParser.java +++ b/core/src/main/java/org/apache/iceberg/SchemaParser.java @@ -49,6 +49,8 @@ private SchemaParser() {} private static final String DOC = "doc"; private static final String NAME = "name"; private static final String ID = "id"; + private static final String INITIAL_DEFAULT = "initial-default"; + private static final String WRITE_DEFAULT = "write-default"; private static final String ELEMENT_ID = "element-id"; private static final String KEY_ID = "key-id"; private static final String VALUE_ID = "value-id"; @@ -88,6 +90,17 @@ private static void toJson( if (field.doc() != null) { generator.writeStringField(DOC, field.doc()); } + + if (field.initialDefault() != null) { + generator.writeFieldName(INITIAL_DEFAULT); + SingleValueParser.toJson(field.type(), field.initialDefault(), generator); + } + + if (field.writeDefault() != null) { + generator.writeFieldName(WRITE_DEFAULT); + SingleValueParser.toJson(field.type(), field.writeDefault(), generator); + } + generator.writeEndObject(); } generator.writeEndArray(); @@ -184,6 +197,22 @@ private static Type typeFromJson(JsonNode json) { throw new IllegalArgumentException("Cannot parse type from json: " + json); } + private static Object defaultFromJson(String defaultField, Type type, JsonNode json) { + if (json.has(defaultField)) { + return SingleValueParser.fromJson(type, json.get(defaultField)); + } + + return null; + } + + private static Types.NestedField.Builder fieldBuilder(boolean isRequired, String name) { + if (isRequired) { + return Types.NestedField.required(name); + } else { + return Types.NestedField.optional(name); + } + } + private static Types.StructType structFromJson(JsonNode json) { JsonNode fieldArray = JsonUtil.get(FIELDS, json); Preconditions.checkArgument( @@ -200,13 +229,19 @@ private static Types.StructType structFromJson(JsonNode json) { String name = JsonUtil.getString(NAME, field); Type type = typeFromJson(JsonUtil.get(TYPE, field)); + Object initialDefault = defaultFromJson(INITIAL_DEFAULT, type, field); + Object writeDefault = defaultFromJson(WRITE_DEFAULT, type, field); + String doc = JsonUtil.getStringOrNull(DOC, field); boolean isRequired = JsonUtil.getBool(REQUIRED, field); - if (isRequired) { - fields.add(Types.NestedField.required(id, name, type, doc)); - } else { - fields.add(Types.NestedField.optional(id, name, type, doc)); - } + fields.add( + fieldBuilder(isRequired, name) + .withId(id) + .ofType(type) + .withDoc(doc) + .withInitialDefault(initialDefault) + .withWriteDefault(writeDefault) + .build()); } return Types.StructType.of(fields); diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaParser.java b/core/src/test/java/org/apache/iceberg/TestSchemaParser.java new file mode 100644 index 000000000000..ebd197a68af0 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestSchemaParser.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.UUID; +import java.util.stream.Stream; +import org.apache.iceberg.avro.AvroDataTest; +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.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +public class TestSchemaParser extends AvroDataTest { + @Override + protected void writeAndValidate(Schema schema) throws IOException { + Schema serialized = SchemaParser.fromJson(SchemaParser.toJson(schema)); + assertThat(serialized.asStruct()).isEqualTo(schema.asStruct()); + } + + @Test + public void testSchemaId() { + Schema schema = new Schema(34, required(1, "id", Types.LongType.get())); + + Schema serialized = SchemaParser.fromJson(SchemaParser.toJson(schema)); + assertThat(serialized.schemaId()).isEqualTo(schema.schemaId()); + } + + @Test + public void testIdentifierColumns() { + Schema schema = + new Schema( + Lists.newArrayList( + required(1, "id-1", Types.LongType.get()), + required(2, "id-2", Types.LongType.get()), + optional(3, "data", Types.StringType.get())), + Sets.newHashSet(1, 2)); + + Schema serialized = SchemaParser.fromJson(SchemaParser.toJson(schema)); + assertThat(serialized.identifierFieldIds()).isEqualTo(Sets.newHashSet(1, 2)); + } + + @Test + public void testDocStrings() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get(), "unique identifier"), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withDoc("payload") + .build()); + + Schema serialized = SchemaParser.fromJson(SchemaParser.toJson(schema)); + assertThat(serialized.findField("id").doc()).isEqualTo("unique identifier"); + assertThat(serialized.findField("data").doc()).isEqualTo("payload"); + } + + private static Stream primitiveTypesAndDefaults() { + return Stream.of( + Arguments.of(Types.BooleanType.get(), false), + Arguments.of(Types.IntegerType.get(), 34), + Arguments.of(Types.LongType.get(), 4900000000L), + Arguments.of(Types.FloatType.get(), 12.21F), + Arguments.of(Types.DoubleType.get(), -0.0D), + Arguments.of(Types.DateType.get(), DateTimeUtil.isoDateToDays("2024-12-17")), + // Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), + Arguments.of( + Types.TimestampType.withZone(), + DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00")), + Arguments.of( + Types.TimestampType.withoutZone(), + DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999")), + Arguments.of(Types.StringType.get(), "iceberg"), + Arguments.of(Types.UUIDType.get(), UUID.randomUUID()), + Arguments.of( + Types.FixedType.ofLength(4), ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), + Arguments.of(Types.BinaryType.get(), ByteBuffer.wrap(new byte[] {0x0a, 0x0b})), + Arguments.of(Types.DecimalType.of(9, 2), new BigDecimal("12.34"))); + } + + @ParameterizedTest + @MethodSource("primitiveTypesAndDefaults") + public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defaultValue) { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.required("col_with_default") + .withId(2) + .ofType(type) + .withInitialDefault(defaultValue) + .withWriteDefault(defaultValue) + .build()); + + Schema serialized = SchemaParser.fromJson(SchemaParser.toJson(schema)); + assertThat(serialized.findField("col_with_default").initialDefault()).isEqualTo(defaultValue); + assertThat(serialized.findField("col_with_default").writeDefault()).isEqualTo(defaultValue); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java index eb97185e21f1..2d51992dd96a 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java @@ -77,7 +77,7 @@ private CloseableIterable newAvroIterable( .reuseContainers() .project(projection) .split(start, length) - .createReaderFunc(readSchema -> SparkPlannedAvroReader.create(projection, idToConstant)) + .createResolvingReader(schema -> SparkPlannedAvroReader.create(schema, idToConstant)) .withNameMapping(nameMapping()) .build(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java index d6e8ae773b4b..ad969384c586 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java @@ -27,13 +27,11 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.file.Path; -import java.util.Map; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Stream; import org.apache.iceberg.Schema; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; @@ -42,8 +40,8 @@ import org.apache.iceberg.types.Types.MapType; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.DateTimeUtil; -import org.apache.spark.sql.internal.SQLConf; import org.assertj.core.api.Assumptions; +import org.assertj.core.api.Condition; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; @@ -285,8 +283,13 @@ public void testMissingRequiredWithoutDefault() { .build()); assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Missing required field: missing_str"); + .has( + new Condition<>( + t -> + IllegalArgumentException.class.isInstance(t) + || IllegalArgumentException.class.isInstance(t.getCause()), + "Expecting a throwable or cause that is an instance of IllegalArgumentException")) + .hasMessageContaining("Missing required field: missing_str"); } @Test @@ -542,44 +545,4 @@ public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Object defau writeAndValidate(writeSchema, readSchema); } - - protected void withSQLConf(Map conf, Action action) throws IOException { - SQLConf sqlConf = SQLConf.get(); - - Map currentConfValues = Maps.newHashMap(); - conf.keySet() - .forEach( - confKey -> { - if (sqlConf.contains(confKey)) { - String currentConfValue = sqlConf.getConfString(confKey); - currentConfValues.put(confKey, currentConfValue); - } - }); - - conf.forEach( - (confKey, confValue) -> { - if (SQLConf.isStaticConfigKey(confKey)) { - throw new RuntimeException("Cannot modify the value of a static config: " + confKey); - } - sqlConf.setConfString(confKey, confValue); - }); - - try { - action.invoke(); - } finally { - conf.forEach( - (confKey, confValue) -> { - if (currentConfValues.containsKey(confKey)) { - sqlConf.setConfString(confKey, currentConfValues.get(confKey)); - } else { - sqlConf.unsetConf(confKey); - } - }); - } - } - - @FunctionalInterface - protected interface Action { - void invoke() throws IOException; - } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/ParameterizedAvroDataTest.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/ParameterizedAvroDataTest.java deleted file mode 100644 index 85effe7d39a7..000000000000 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/ParameterizedAvroDataTest.java +++ /dev/null @@ -1,284 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.data; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.IOException; -import java.nio.file.Path; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.types.Types.ListType; -import org.apache.iceberg.types.Types.LongType; -import org.apache.iceberg.types.Types.MapType; -import org.apache.iceberg.types.Types.StructType; -import org.apache.spark.sql.internal.SQLConf; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.io.TempDir; - -/** - * Copy of {@link AvroDataTest} that marks tests with @{@link org.junit.jupiter.api.TestTemplate} - * instead of @{@link Test} to make the tests work in a parameterized environment. - */ -public abstract class ParameterizedAvroDataTest { - - protected abstract void writeAndValidate(Schema schema) throws IOException; - - protected static final StructType SUPPORTED_PRIMITIVES = - StructType.of( - required(100, "id", LongType.get()), - optional(101, "data", Types.StringType.get()), - required(102, "b", Types.BooleanType.get()), - optional(103, "i", Types.IntegerType.get()), - required(104, "l", LongType.get()), - optional(105, "f", Types.FloatType.get()), - required(106, "d", Types.DoubleType.get()), - optional(107, "date", Types.DateType.get()), - required(108, "ts", Types.TimestampType.withZone()), - required(110, "s", Types.StringType.get()), - required(111, "uuid", Types.UUIDType.get()), - required(112, "fixed", Types.FixedType.ofLength(7)), - optional(113, "bytes", Types.BinaryType.get()), - required(114, "dec_9_0", Types.DecimalType.of(9, 0)), // int encoded - required(115, "dec_11_2", Types.DecimalType.of(11, 2)), // long encoded - required(116, "dec_20_5", Types.DecimalType.of(20, 5)), // requires padding - required(117, "dec_38_10", Types.DecimalType.of(38, 10)) // Spark's maximum precision - ); - - @TempDir protected Path temp; - - @TestTemplate - public void testSimpleStruct() throws IOException { - writeAndValidate(TypeUtil.assignIncreasingFreshIds(new Schema(SUPPORTED_PRIMITIVES.fields()))); - } - - @TestTemplate - public void testStructWithRequiredFields() throws IOException { - writeAndValidate( - TypeUtil.assignIncreasingFreshIds( - new Schema( - Lists.transform(SUPPORTED_PRIMITIVES.fields(), Types.NestedField::asRequired)))); - } - - @TestTemplate - public void testStructWithOptionalFields() throws IOException { - writeAndValidate( - TypeUtil.assignIncreasingFreshIds( - new Schema( - Lists.transform(SUPPORTED_PRIMITIVES.fields(), Types.NestedField::asOptional)))); - } - - @TestTemplate - public void testNestedStruct() throws IOException { - writeAndValidate( - TypeUtil.assignIncreasingFreshIds(new Schema(required(1, "struct", SUPPORTED_PRIMITIVES)))); - } - - @TestTemplate - public void testArray() throws IOException { - Schema schema = - new Schema( - required(0, "id", LongType.get()), - optional(1, "data", ListType.ofOptional(2, Types.StringType.get()))); - - writeAndValidate(schema); - } - - @TestTemplate - public void testArrayOfStructs() throws IOException { - Schema schema = - TypeUtil.assignIncreasingFreshIds( - new Schema( - required(0, "id", LongType.get()), - optional(1, "data", ListType.ofOptional(2, SUPPORTED_PRIMITIVES)))); - - writeAndValidate(schema); - } - - @TestTemplate - public void testMap() throws IOException { - Schema schema = - new Schema( - required(0, "id", LongType.get()), - optional( - 1, - "data", - MapType.ofOptional(2, 3, Types.StringType.get(), Types.StringType.get()))); - - writeAndValidate(schema); - } - - @TestTemplate - public void testNumericMapKey() throws IOException { - Schema schema = - new Schema( - required(0, "id", LongType.get()), - optional(1, "data", MapType.ofOptional(2, 3, LongType.get(), Types.StringType.get()))); - - writeAndValidate(schema); - } - - @TestTemplate - public void testComplexMapKey() throws IOException { - Schema schema = - new Schema( - required(0, "id", LongType.get()), - optional( - 1, - "data", - MapType.ofOptional( - 2, - 3, - StructType.of( - required(4, "i", Types.IntegerType.get()), - optional(5, "s", Types.StringType.get())), - Types.StringType.get()))); - - writeAndValidate(schema); - } - - @TestTemplate - public void testMapOfStructs() throws IOException { - Schema schema = - TypeUtil.assignIncreasingFreshIds( - new Schema( - required(0, "id", LongType.get()), - optional( - 1, - "data", - MapType.ofOptional(2, 3, Types.StringType.get(), SUPPORTED_PRIMITIVES)))); - - writeAndValidate(schema); - } - - @TestTemplate - public void testMixedTypes() throws IOException { - StructType structType = - StructType.of( - required(0, "id", LongType.get()), - optional( - 1, - "list_of_maps", - ListType.ofOptional( - 2, MapType.ofOptional(3, 4, Types.StringType.get(), SUPPORTED_PRIMITIVES))), - optional( - 5, - "map_of_lists", - MapType.ofOptional( - 6, 7, Types.StringType.get(), ListType.ofOptional(8, SUPPORTED_PRIMITIVES))), - required( - 9, - "list_of_lists", - ListType.ofOptional(10, ListType.ofOptional(11, SUPPORTED_PRIMITIVES))), - required( - 12, - "map_of_maps", - MapType.ofOptional( - 13, - 14, - Types.StringType.get(), - MapType.ofOptional(15, 16, Types.StringType.get(), SUPPORTED_PRIMITIVES))), - required( - 17, - "list_of_struct_of_nested_types", - ListType.ofOptional( - 19, - StructType.of( - Types.NestedField.required( - 20, - "m1", - MapType.ofOptional( - 21, 22, Types.StringType.get(), SUPPORTED_PRIMITIVES)), - Types.NestedField.optional( - 23, "l1", ListType.ofRequired(24, SUPPORTED_PRIMITIVES)), - Types.NestedField.required( - 25, "l2", ListType.ofRequired(26, SUPPORTED_PRIMITIVES)), - Types.NestedField.optional( - 27, - "m2", - MapType.ofOptional( - 28, 29, Types.StringType.get(), SUPPORTED_PRIMITIVES)))))); - - Schema schema = - new Schema( - TypeUtil.assignFreshIds(structType, new AtomicInteger(0)::incrementAndGet) - .asStructType() - .fields()); - - writeAndValidate(schema); - } - - @TestTemplate - public void testTimestampWithoutZone() throws IOException { - Schema schema = - TypeUtil.assignIncreasingFreshIds( - new Schema( - required(0, "id", LongType.get()), - optional(1, "ts_without_zone", Types.TimestampType.withoutZone()))); - - writeAndValidate(schema); - } - - protected void withSQLConf(Map conf, Action action) throws IOException { - SQLConf sqlConf = SQLConf.get(); - - Map currentConfValues = Maps.newHashMap(); - conf.keySet() - .forEach( - confKey -> { - if (sqlConf.contains(confKey)) { - String currentConfValue = sqlConf.getConfString(confKey); - currentConfValues.put(confKey, currentConfValue); - } - }); - - conf.forEach( - (confKey, confValue) -> { - if (SQLConf.isStaticConfigKey(confKey)) { - throw new RuntimeException("Cannot modify the value of a static config: " + confKey); - } - sqlConf.setConfString(confKey, confValue); - }); - - try { - action.invoke(); - } finally { - conf.forEach( - (confKey, confValue) -> { - if (currentConfValues.containsKey(confKey)) { - sqlConf.setConfString(confKey, currentConfValues.get(confKey)); - } else { - sqlConf.unsetConf(confKey); - } - }); - } - } - - @FunctionalInterface - protected interface Action { - void invoke() throws IOException; - } -} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 64d0b85625a9..a0e77e2acbae 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -96,11 +96,20 @@ public static void assertEqualsSafe(Types.StructType struct, List recs, public static void assertEqualsSafe(Types.StructType struct, Record rec, Row row) { List fields = struct.fields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i).type(); + for (int readPos = 0; readPos < fields.size(); readPos += 1) { + Types.NestedField field = fields.get(readPos); + Field writeField = rec.getSchema().getField(field.name()); - Object expectedValue = rec.get(i); - Object actualValue = row.get(i); + Type fieldType = field.type(); + Object actualValue = row.get(readPos); + + Object expectedValue; + if (writeField != null) { + int writePos = writeField.pos(); + expectedValue = rec.get(writePos); + } else { + expectedValue = field.initialDefault(); + } assertEqualsSafe(fieldType, expectedValue, actualValue); } @@ -237,11 +246,21 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) .isEqualTo(String.valueOf(expected)); break; case FIXED: - assertThat(expected).as("Should expect a Fixed").isInstanceOf(GenericData.Fixed.class); + // generated data is written using Avro or Parquet/Avro so generated rows use + // GenericData.Fixed, but default values are converted from Iceberg's internal + // representation so the expected value may be either class. + byte[] expectedBytes; + if (expected instanceof ByteBuffer) { + expectedBytes = ByteBuffers.toByteArray((ByteBuffer) expected); + } else if (expected instanceof GenericData.Fixed) { + expectedBytes = ((GenericData.Fixed) expected).bytes(); + } else { + throw new IllegalStateException( + "Invalid expected value, not byte[] or Fixed: " + expected); + } + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); - assertThat(actual) - .as("Bytes should match") - .isEqualTo(((GenericData.Fixed) expected).bytes()); + assertThat(actual).as("Bytes should match").isEqualTo(expectedBytes); break; case BINARY: assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java new file mode 100644 index 000000000000..756370eec0da --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Iterator; +import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.Tables; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkPlannedAvroReader; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.DataFrameWriter; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public abstract class DataFrameWriteTestBase extends ScanTestBase { + @TempDir private Path temp; + + @Override + protected boolean supportsDefaultValues() { + // disable default value tests because this tests the write path + return false; + } + + @Override + protected void writeRecords(Table table, List records) throws IOException { + Schema tableSchema = table.schema(); // use the table schema because ids are reassigned + + Dataset df = createDataset(records, tableSchema); + DataFrameWriter writer = df.write().format("iceberg").mode("append"); + + writer.save(table.location()); + + // refresh the in-memory table state to pick up Spark's write + table.refresh(); + } + + private Dataset createDataset(List records, Schema schema) + throws IOException { + // this uses the SparkAvroReader to create a DataFrame from the list of records + // it assumes that SparkAvroReader is correct + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)).schema(schema).named("test").build()) { + for (GenericData.Record rec : records) { + writer.add(rec); + } + } + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createResolvingReader(SparkPlannedAvroReader::create) + .project(schema) + .build()) { + rows = Lists.newArrayList(reader); + } + + // verify that the dataframe matches + assertThat(rows.size()).isEqualTo(records.size()); + Iterator recordIter = records.iterator(); + for (InternalRow row : rows) { + assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); + } + + JavaRDD rdd = sc.parallelize(rows); + return spark.internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); + } + + @Test + public void testAlternateLocation() throws IOException { + Schema schema = new Schema(required(1, "id", Types.LongType.get())); + + File location = temp.resolve("table_location").toFile(); + File altLocation = temp.resolve("alt_location").toFile(); + + Tables tables = new HadoopTables(spark.sessionState().newHadoopConf()); + Table table = tables.create(schema, PartitionSpec.unpartitioned(), location.toString()); + + // override the table's data location + table + .updateProperties() + .set(TableProperties.WRITE_DATA_LOCATION, altLocation.getAbsolutePath()) + .commit(); + + writeRecords(table, RandomData.generateList(table.schema(), 100, 87112L)); + + table + .currentSnapshot() + .addedDataFiles(table.io()) + .forEach( + dataFile -> + assertThat(dataFile.location()) + .as( + String.format( + "File should have the parent directory %s, but has: %s.", + altLocation, dataFile.location())) + .startsWith(altLocation + "/")); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java new file mode 100644 index 000000000000..3a269740b709 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.spark.data.AvroDataTest; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.types.TypeUtil; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; + +/** An AvroDataScan test that validates data by reading through Spark */ +public abstract class ScanTestBase extends AvroDataTest { + private static final Configuration CONF = new Configuration(); + + protected static SparkSession spark = null; + protected static JavaSparkContext sc = null; + + @BeforeAll + public static void startSpark() { + ScanTestBase.spark = SparkSession.builder().master("local[2]").getOrCreate(); + ScanTestBase.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = ScanTestBase.spark; + ScanTestBase.spark = null; + ScanTestBase.sc = null; + currentSpark.stop(); + } + + @TempDir private Path temp; + + protected void configureTable(Table table) {} + + protected abstract void writeRecords(Table table, List records) + throws IOException; + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(schema, schema); + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + File parent = temp.resolve("scan_test").toFile(); + File location = new File(parent, "test"); + + HadoopTables tables = new HadoopTables(CONF); + Table table = tables.create(writeSchema, PartitionSpec.unpartitioned(), location.toString()); + + // Important: use the table's schema for the rest of the test + // When tables are created, the column ids are reassigned. + List expected = RandomData.generateList(table.schema(), 100, 1L); + + writeRecords(table, expected); + + // update the table schema to the expected schema + if (!expectedSchema.sameSchema(table.schema())) { + Schema expectedSchemaWithTableIds = + TypeUtil.reassignOrRefreshIds(expectedSchema, table.schema()); + int highestFieldId = + Math.max(table.schema().highestFieldId(), expectedSchema.highestFieldId()); + + // don't use the table API because tests cover incompatible update cases + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata builder = + TableMetadata.buildFrom(ops.current()) + .upgradeFormatVersion(3) + .setCurrentSchema(expectedSchemaWithTableIds, highestFieldId) + .build(); + ops.commit(ops.current(), builder); + } + + Dataset df = spark.read().format("iceberg").load(table.location()); + + List rows = df.collectAsList(); + assertThat(rows).as("Should contain 100 rows").hasSize(100); + + for (int i = 0; i < expected.size(); i += 1) { + TestHelpers.assertEqualsSafe(table.schema().asStruct(), expected.get(i), rows.get(i)); + } + } + + @Override + protected boolean supportsDefaultValues() { + return true; + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java new file mode 100644 index 000000000000..110428d0a20c --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; + +public class TestAvroDataFrameWrite extends DataFrameWriteTestBase { + @Override + protected void configureTable(Table table) { + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.AVRO.toString()) + .commit(); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java index 8345a4e0a697..96627c7f7334 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java @@ -19,93 +19,41 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.Files.localOutput; -import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; -import java.nio.file.Path; import java.util.List; import java.util.UUID; import org.apache.avro.generic.GenericData.Record; -import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.spark.data.AvroDataTest; -import org.apache.iceberg.spark.data.RandomData; -import org.apache.iceberg.spark.data.TestHelpers; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.io.TempDir; - -public class TestAvroScan extends AvroDataTest { - private static final Configuration CONF = new Configuration(); - - @TempDir private Path temp; - - private static SparkSession spark = null; - - @BeforeAll - public static void startSpark() { - TestAvroScan.spark = SparkSession.builder().master("local[2]").getOrCreate(); - } - - @AfterAll - public static void stopSpark() { - SparkSession currentSpark = TestAvroScan.spark; - TestAvroScan.spark = null; - currentSpark.stop(); - } +public class TestAvroScan extends ScanTestBase { @Override - protected void writeAndValidate(Schema schema) throws IOException { - File parent = temp.resolve("avro").toFile(); - File location = new File(parent, "test"); - File dataFolder = new File(location, "data"); + protected void writeRecords(Table table, List records) throws IOException { + File dataFolder = new File(table.location(), "data"); dataFolder.mkdirs(); File avroFile = new File(dataFolder, FileFormat.AVRO.addExtension(UUID.randomUUID().toString())); - HadoopTables tables = new HadoopTables(CONF); - Table table = tables.create(schema, PartitionSpec.unpartitioned(), location.toString()); - - // Important: use the table's schema for the rest of the test - // When tables are created, the column ids are reassigned. - Schema tableSchema = table.schema(); - - List expected = RandomData.generateList(tableSchema, 100, 1L); - try (FileAppender writer = - Avro.write(localOutput(avroFile)).schema(tableSchema).build()) { - writer.addAll(expected); + Avro.write(localOutput(avroFile)).schema(table.schema()).build()) { + writer.addAll(records); } DataFile file = DataFiles.builder(PartitionSpec.unpartitioned()) - .withRecordCount(100) .withFileSizeInBytes(avroFile.length()) + .withRecordCount(records.size()) .withPath(avroFile.toString()) .build(); table.newAppend().appendFile(file).commit(); - - Dataset df = spark.read().format("iceberg").load(location.toString()); - - List rows = df.collectAsList(); - assertThat(rows).as("Should contain 100 rows").hasSize(100); - - for (int i = 0; i < expected.size(); i += 1) { - TestHelpers.assertEqualsSafe(tableSchema.asStruct(), expected.get(i), rows.get(i)); - } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java deleted file mode 100644 index 42552f385137..000000000000 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ /dev/null @@ -1,412 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.source; - -import static org.apache.iceberg.spark.SparkSchemaUtil.convert; -import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsSafe; -import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; - -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.util.Arrays; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Random; -import org.apache.avro.generic.GenericData.Record; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.Files; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.avro.AvroIterable; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.FileAppender; -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.SparkSQLProperties; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.SparkWriteOptions; -import org.apache.iceberg.spark.data.ParameterizedAvroDataTest; -import org.apache.iceberg.spark.data.RandomData; -import org.apache.iceberg.spark.data.SparkPlannedAvroReader; -import org.apache.iceberg.types.Types; -import org.apache.spark.SparkException; -import org.apache.spark.TaskContext; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.MapPartitionsFunction; -import org.apache.spark.sql.DataFrameWriter; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.InternalRow; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.io.TempDir; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestDataFrameWrites extends ParameterizedAvroDataTest { - private static final Configuration CONF = new Configuration(); - - @Parameters(name = "format = {0}") - public static Collection parameters() { - return Arrays.asList("parquet", "avro", "orc"); - } - - @Parameter private String format; - - @TempDir private File location; - - private static SparkSession spark = null; - private static JavaSparkContext sc = null; - - private Map tableProperties; - - private final org.apache.spark.sql.types.StructType sparkSchema = - new org.apache.spark.sql.types.StructType( - new org.apache.spark.sql.types.StructField[] { - new org.apache.spark.sql.types.StructField( - "optionalField", - org.apache.spark.sql.types.DataTypes.StringType, - true, - org.apache.spark.sql.types.Metadata.empty()), - new org.apache.spark.sql.types.StructField( - "requiredField", - org.apache.spark.sql.types.DataTypes.StringType, - false, - org.apache.spark.sql.types.Metadata.empty()) - }); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.optional(1, "optionalField", Types.StringType.get()), - Types.NestedField.required(2, "requiredField", Types.StringType.get())); - - private final List data0 = - Arrays.asList( - "{\"optionalField\": \"a1\", \"requiredField\": \"bid_001\"}", - "{\"optionalField\": \"a2\", \"requiredField\": \"bid_002\"}"); - private final List data1 = - Arrays.asList( - "{\"optionalField\": \"d1\", \"requiredField\": \"bid_101\"}", - "{\"optionalField\": \"d2\", \"requiredField\": \"bid_102\"}", - "{\"optionalField\": \"d3\", \"requiredField\": \"bid_103\"}", - "{\"optionalField\": \"d4\", \"requiredField\": \"bid_104\"}"); - - @BeforeAll - public static void startSpark() { - TestDataFrameWrites.spark = SparkSession.builder().master("local[2]").getOrCreate(); - TestDataFrameWrites.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - } - - @AfterAll - public static void stopSpark() { - SparkSession currentSpark = TestDataFrameWrites.spark; - TestDataFrameWrites.spark = null; - TestDataFrameWrites.sc = null; - currentSpark.stop(); - } - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - Table table = createTable(schema); - writeAndValidateWithLocations(table, new File(location, "data")); - } - - @TestTemplate - public void testWriteWithCustomDataLocation() throws IOException { - File tablePropertyDataLocation = temp.resolve("test-table-property-data-dir").toFile(); - Table table = createTable(new Schema(SUPPORTED_PRIMITIVES.fields())); - table - .updateProperties() - .set(TableProperties.WRITE_DATA_LOCATION, tablePropertyDataLocation.getAbsolutePath()) - .commit(); - writeAndValidateWithLocations(table, tablePropertyDataLocation); - } - - private Table createTable(Schema schema) { - HadoopTables tables = new HadoopTables(CONF); - return tables.create(schema, PartitionSpec.unpartitioned(), location.toString()); - } - - private void writeAndValidateWithLocations(Table table, File expectedDataDir) throws IOException { - Schema tableSchema = table.schema(); // use the table schema because ids are reassigned - - table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit(); - - Iterable expected = RandomData.generate(tableSchema, 100, 0L); - writeData(expected, tableSchema); - - table.refresh(); - - List actual = readTable(); - - Iterator expectedIter = expected.iterator(); - Iterator actualIter = actual.iterator(); - while (expectedIter.hasNext() && actualIter.hasNext()) { - assertEqualsSafe(tableSchema.asStruct(), expectedIter.next(), actualIter.next()); - } - assertThat(actualIter.hasNext()) - .as("Both iterators should be exhausted") - .isEqualTo(expectedIter.hasNext()); - - table - .currentSnapshot() - .addedDataFiles(table.io()) - .forEach( - dataFile -> - assertThat(URI.create(dataFile.location()).getPath()) - .as( - String.format( - "File should have the parent directory %s, but has: %s.", - expectedDataDir.getAbsolutePath(), dataFile.location())) - .startsWith(expectedDataDir.getAbsolutePath())); - } - - private List readTable() { - Dataset result = spark.read().format("iceberg").load(location.toString()); - - return result.collectAsList(); - } - - private void writeData(Iterable records, Schema schema) throws IOException { - Dataset df = createDataset(records, schema); - DataFrameWriter writer = df.write().format("iceberg").mode("append"); - writer.save(location.toString()); - } - - private void writeDataWithFailOnPartition(Iterable records, Schema schema) - throws IOException, SparkException { - final int numPartitions = 10; - final int partitionToFail = new Random().nextInt(numPartitions); - MapPartitionsFunction failOnFirstPartitionFunc = - input -> { - int partitionId = TaskContext.getPartitionId(); - - if (partitionId == partitionToFail) { - throw new SparkException( - String.format("Intended exception in partition %d !", partitionId)); - } - return input; - }; - - Dataset df = - createDataset(records, schema) - .repartition(numPartitions) - .mapPartitions(failOnFirstPartitionFunc, Encoders.row(convert(schema))); - // This trick is needed because Spark 3 handles decimal overflow in RowEncoder which "changes" - // nullability of the column to "true" regardless of original nullability. - // Setting "check-nullability" option to "false" doesn't help as it fails at Spark analyzer. - Dataset convertedDf = df.sqlContext().createDataFrame(df.rdd(), convert(schema)); - DataFrameWriter writer = convertedDf.write().format("iceberg").mode("append"); - writer.save(location.toString()); - } - - private Dataset createDataset(Iterable records, Schema schema) throws IOException { - // this uses the SparkAvroReader to create a DataFrame from the list of records - // it assumes that SparkAvroReader is correct - File testFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(testFile.delete()).as("Delete should succeed").isTrue(); - - try (FileAppender writer = - Avro.write(Files.localOutput(testFile)).schema(schema).named("test").build()) { - for (Record rec : records) { - writer.add(rec); - } - } - - // make sure the dataframe matches the records before moving on - List rows = Lists.newArrayList(); - try (AvroIterable reader = - Avro.read(Files.localInput(testFile)) - .createResolvingReader(SparkPlannedAvroReader::create) - .project(schema) - .build()) { - - Iterator recordIter = records.iterator(); - Iterator readIter = reader.iterator(); - while (recordIter.hasNext() && readIter.hasNext()) { - InternalRow row = readIter.next(); - assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); - rows.add(row); - } - assertThat(readIter.hasNext()) - .as("Both iterators should be exhausted") - .isEqualTo(recordIter.hasNext()); - } - - JavaRDD rdd = sc.parallelize(rows); - return spark.internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); - } - - @TestTemplate - public void testNullableWithWriteOption() throws IOException { - assumeThat(spark.version()) - .as("Spark 3 rejects writing nulls to a required column") - .startsWith("2"); - - String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location); - String targetPath = String.format("%s/nullable_poc/targetFolder/", location); - - tableProperties = ImmutableMap.of(TableProperties.WRITE_DATA_LOCATION, targetPath); - - // read this and append to iceberg dataset - spark - .read() - .schema(sparkSchema) - .json(JavaSparkContext.fromSparkContext(spark.sparkContext()).parallelize(data1)) - .write() - .parquet(sourcePath); - - // this is our iceberg dataset to which we will append data - new HadoopTables(spark.sessionState().newHadoopConf()) - .create( - icebergSchema, - PartitionSpec.builderFor(icebergSchema).identity("requiredField").build(), - tableProperties, - targetPath); - - // this is the initial data inside the iceberg dataset - spark - .read() - .schema(sparkSchema) - .json(JavaSparkContext.fromSparkContext(spark.sparkContext()).parallelize(data0)) - .write() - .format("iceberg") - .mode(SaveMode.Append) - .save(targetPath); - - // read from parquet and append to iceberg w/ nullability check disabled - spark - .read() - .schema(SparkSchemaUtil.convert(icebergSchema)) - .parquet(sourcePath) - .write() - .format("iceberg") - .option(SparkWriteOptions.CHECK_NULLABILITY, false) - .mode(SaveMode.Append) - .save(targetPath); - - // read all data - List rows = spark.read().format("iceberg").load(targetPath).collectAsList(); - assumeThat(rows).as("Should contain 6 rows").hasSize(6); - } - - @TestTemplate - public void testNullableWithSparkSqlOption() throws IOException { - assumeThat(spark.version()) - .as("Spark 3 rejects writing nulls to a required column") - .startsWith("2"); - - String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location); - String targetPath = String.format("%s/nullable_poc/targetFolder/", location); - - tableProperties = ImmutableMap.of(TableProperties.WRITE_DATA_LOCATION, targetPath); - - // read this and append to iceberg dataset - spark - .read() - .schema(sparkSchema) - .json(JavaSparkContext.fromSparkContext(spark.sparkContext()).parallelize(data1)) - .write() - .parquet(sourcePath); - - SparkSession newSparkSession = - SparkSession.builder() - .master("local[2]") - .appName("NullableTest") - .config(SparkSQLProperties.CHECK_NULLABILITY, false) - .getOrCreate(); - - // this is our iceberg dataset to which we will append data - new HadoopTables(newSparkSession.sessionState().newHadoopConf()) - .create( - icebergSchema, - PartitionSpec.builderFor(icebergSchema).identity("requiredField").build(), - tableProperties, - targetPath); - - // this is the initial data inside the iceberg dataset - newSparkSession - .read() - .schema(sparkSchema) - .json(JavaSparkContext.fromSparkContext(spark.sparkContext()).parallelize(data0)) - .write() - .format("iceberg") - .mode(SaveMode.Append) - .save(targetPath); - - // read from parquet and append to iceberg - newSparkSession - .read() - .schema(SparkSchemaUtil.convert(icebergSchema)) - .parquet(sourcePath) - .write() - .format("iceberg") - .mode(SaveMode.Append) - .save(targetPath); - - // read all data - List rows = newSparkSession.read().format("iceberg").load(targetPath).collectAsList(); - assumeThat(rows).as("Should contain 6 rows").hasSize(6); - } - - @TestTemplate - public void testFaultToleranceOnWrite() throws IOException { - Schema schema = new Schema(SUPPORTED_PRIMITIVES.fields()); - Table table = createTable(schema); - - Iterable records = RandomData.generate(schema, 100, 0L); - writeData(records, schema); - - table.refresh(); - - Snapshot snapshotBeforeFailingWrite = table.currentSnapshot(); - List resultBeforeFailingWrite = readTable(); - - Iterable records2 = RandomData.generate(schema, 100, 0L); - - assertThatThrownBy(() -> writeDataWithFailOnPartition(records2, schema)) - .isInstanceOf(SparkException.class); - - table.refresh(); - - Snapshot snapshotAfterFailingWrite = table.currentSnapshot(); - List resultAfterFailingWrite = readTable(); - - assertThat(snapshotBeforeFailingWrite).isEqualTo(snapshotAfterFailingWrite); - assertThat(resultBeforeFailingWrite).isEqualTo(resultAfterFailingWrite); - } -} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java new file mode 100644 index 000000000000..35be6423ee23 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; + +public class TestORCDataFrameWrite extends DataFrameWriteTestBase { + @Override + protected void configureTable(Table table) { + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.ORC.toString()) + .commit(); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java new file mode 100644 index 000000000000..90a9ac48a486 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; + +public class TestParquetDataFrameWrite extends DataFrameWriteTestBase { + @Override + protected void configureTable(Table table) { + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.toString()) + .commit(); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java index ebeed62acce4..c0dee43d6de1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java @@ -19,140 +19,41 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.Files.localOutput; -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; -import static org.apache.spark.sql.functions.monotonically_increasing_id; -import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; -import java.nio.file.Path; -import java.util.Arrays; -import java.util.Collection; import java.util.List; import java.util.UUID; import org.apache.avro.generic.GenericData; -import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.spark.data.ParameterizedAvroDataTest; -import org.apache.iceberg.spark.data.RandomData; -import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.io.TempDir; -@ExtendWith(ParameterizedTestExtension.class) -public class TestParquetScan extends ParameterizedAvroDataTest { - private static final Configuration CONF = new Configuration(); - - private static SparkSession spark = null; - - @BeforeAll - public static void startSpark() { - TestParquetScan.spark = SparkSession.builder().master("local[2]").getOrCreate(); - } - - @AfterAll - public static void stopSpark() { - SparkSession currentSpark = TestParquetScan.spark; - TestParquetScan.spark = null; - currentSpark.stop(); - } - - @TempDir private Path temp; - - @Parameter private boolean vectorized; - - @Parameters(name = "vectorized = {0}") - public static Collection parameters() { - return Arrays.asList(false, true); +public class TestParquetScan extends ScanTestBase { + protected boolean vectorized() { + return false; } @Override - protected void writeAndValidate(Schema schema) throws IOException { - assumeThat( - TypeUtil.find( - schema, - type -> type.isMapType() && type.asMapType().keyType() != Types.StringType.get())) - .as("Cannot handle non-string map keys in parquet-avro") - .isNull(); - - assertThat(vectorized).as("should not be null").isNotNull(); - Table table = createTable(schema); - - // Important: use the table's schema for the rest of the test - // When tables are created, the column ids are reassigned. - List expected = RandomData.generateList(table.schema(), 100, 1L); - writeRecords(table, expected); - - configureVectorization(table); - - Dataset df = spark.read().format("iceberg").load(table.location()); - - List rows = df.collectAsList(); - assertThat(rows).as("Should contain 100 rows").hasSize(100); - - for (int i = 0; i < expected.size(); i += 1) { - TestHelpers.assertEqualsSafe(table.schema().asStruct(), expected.get(i), rows.get(i)); - } - } - - @TestTemplate - public void testEmptyTableProjection() throws IOException { - Types.StructType structType = - Types.StructType.of( - required(100, "id", Types.LongType.get()), - optional(101, "data", Types.StringType.get()), - required(102, "b", Types.BooleanType.get()), - optional(103, "i", Types.IntegerType.get())); - Table table = createTable(new Schema(structType.fields())); - - List expected = RandomData.generateList(table.schema(), 100, 1L); - writeRecords(table, expected); - - configureVectorization(table); - - List rows = - spark - .read() - .format("iceberg") - .load(table.location()) - .select(monotonically_increasing_id()) - .collectAsList(); - assertThat(rows).hasSize(100); - } - - private Table createTable(Schema schema) throws IOException { - File parent = temp.resolve("parquet").toFile(); - File location = new File(parent, "test"); - HadoopTables tables = new HadoopTables(CONF); - return tables.create(schema, PartitionSpec.unpartitioned(), location.toString()); + protected void configureTable(Table table) { + table + .updateProperties() + .set(TableProperties.PARQUET_VECTORIZATION_ENABLED, String.valueOf(vectorized())) + .commit(); } - private void writeRecords(Table table, List records) throws IOException { + @Override + protected void writeRecords(Table table, List records) throws IOException { File dataFolder = new File(table.location(), "data"); - dataFolder.mkdirs(); - File parquetFile = new File(dataFolder, FileFormat.PARQUET.addExtension(UUID.randomUUID().toString())); @@ -165,16 +66,21 @@ private void writeRecords(Table table, List records) throws DataFiles.builder(PartitionSpec.unpartitioned()) .withFileSizeInBytes(parquetFile.length()) .withPath(parquetFile.toString()) - .withRecordCount(100) + .withRecordCount(records.size()) .build(); table.newAppend().appendFile(file).commit(); } - private void configureVectorization(Table table) { - table - .updateProperties() - .set(TableProperties.PARQUET_VECTORIZATION_ENABLED, String.valueOf(vectorized)) - .commit(); + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + assumeThat( + TypeUtil.find( + writeSchema, + type -> type.isMapType() && type.asMapType().keyType() != Types.StringType.get())) + .as("Cannot handle non-string map keys in parquet-avro") + .isNull(); + + super.writeAndValidate(writeSchema, expectedSchema); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java new file mode 100644 index 000000000000..a6b5166b3a4e --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +public class TestParquetVectorizedScan extends TestParquetScan { + @Override + protected boolean vectorized() { + return true; + } +} From dbf26d7bf5ba7f85317beea52ac2db98db3a8053 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 21:45:26 +0100 Subject: [PATCH 282/313] Build: Bump datamodel-code-generator from 0.26.3 to 0.26.4 (#11856) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.26.3 to 0.26.4. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.26.3...0.26.4) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index 15cb9d7dbe4e..d2c23b842bb6 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.1 -datamodel-code-generator==0.26.3 +datamodel-code-generator==0.26.4 From 4ceb96d4863be4de27f2758647784241c38c59f2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 21:55:03 +0100 Subject: [PATCH 283/313] Build: Bump mkdocs-awesome-pages-plugin from 2.9.3 to 2.10.0 (#11855) Bumps [mkdocs-awesome-pages-plugin](https://github.com/lukasgeiter/mkdocs-awesome-pages-plugin) from 2.9.3 to 2.10.0. - [Release notes](https://github.com/lukasgeiter/mkdocs-awesome-pages-plugin/releases) - [Commits](https://github.com/lukasgeiter/mkdocs-awesome-pages-plugin/compare/v2.9.3...v2.10.0) --- updated-dependencies: - dependency-name: mkdocs-awesome-pages-plugin dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index f3b50381d398..689d2c036097 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -mkdocs-awesome-pages-plugin==2.9.3 +mkdocs-awesome-pages-plugin==2.10.0 mkdocs-macros-plugin==1.3.7 mkdocs-material==9.5.48 mkdocs-material-extensions==1.3.1 From b0a119c29cf27dfa658b9d894ce721031f6c89a1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 21:59:32 +0100 Subject: [PATCH 284/313] Build: Bump mkdocs-material from 9.5.48 to 9.5.49 (#11854) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.48 to 9.5.49. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.5.48...9.5.49) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 689d2c036097..4d1be7a4bc80 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.10.0 mkdocs-macros-plugin==1.3.7 -mkdocs-material==9.5.48 +mkdocs-material==9.5.49 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.2 From 5c5d7c9b74626b8a66ef26fcbdf3d8030c9b33e9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 22:05:03 +0100 Subject: [PATCH 285/313] Build: Bump io.netty:netty-buffer from 4.1.115.Final to 4.1.116.Final (#11853) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.115.Final to 4.1.116.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.115.Final...netty-4.1.116.Final) --- updated-dependencies: - dependency-name: io.netty:netty-buffer dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 16bafc7a736a..52788e14ac47 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -71,8 +71,8 @@ microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.101.2" -netty-buffer = "4.1.115.Final" -netty-buffer-compat = "4.1.115.Final" +netty-buffer = "4.1.116.Final" +netty-buffer-compat = "4.1.116.Final" object-client-bundle = "3.3.2" orc = "1.9.5" parquet = "1.15.0" From bbf5d6f84e4ad01c670892c42eb186baca40e6c7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 22:05:32 +0100 Subject: [PATCH 286/313] Build: Bump software.amazon.awssdk:bom from 2.29.34 to 2.29.39 (#11851) Bumps software.amazon.awssdk:bom from 2.29.34 to 2.29.39. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 52788e14ac47..0c291b345d04 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.29.34" +awssdk-bom = "2.29.39" azuresdk-bom = "1.2.30" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" From 556969ad2a5e69b0d2403a55bd7f33794a9a61d3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 22:05:51 +0100 Subject: [PATCH 287/313] Build: Bump guava from 33.3.1-jre to 33.4.0-jre (#11850) Bumps `guava` from 33.3.1-jre to 33.4.0-jre. Updates `com.google.guava:guava` from 33.3.1-jre to 33.4.0-jre - [Release notes](https://github.com/google/guava/releases) - [Commits](https://github.com/google/guava/commits) Updates `com.google.guava:guava-testlib` from 33.3.1-jre to 33.4.0-jre - [Release notes](https://github.com/google/guava/releases) - [Commits](https://github.com/google/guava/commits) --- updated-dependencies: - dependency-name: com.google.guava:guava dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: com.google.guava:guava-testlib dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 0c291b345d04..a12c61219cfa 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -45,7 +45,7 @@ flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} flink120 = { strictly = "1.20.0"} google-libraries-bom = "26.51.0" -guava = "33.3.1-jre" +guava = "33.4.0-jre" hadoop2 = "2.7.3" hadoop3 = "3.4.1" httpcomponents-httpclient5 = "5.4.1" From e0ccebca5b3224919be91bd6df405415742884e7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 22:10:07 +0100 Subject: [PATCH 288/313] Build: Bump junit from 5.11.3 to 5.11.4 (#11849) Bumps `junit` from 5.11.3 to 5.11.4. Updates `org.junit.jupiter:junit-jupiter` from 5.11.3 to 5.11.4 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/compare/r5.11.3...r5.11.4) Updates `org.junit.jupiter:junit-jupiter-engine` from 5.11.3 to 5.11.4 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/compare/r5.11.3...r5.11.4) Updates `org.junit.vintage:junit-vintage-engine` from 5.11.3 to 5.11.4 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/compare/r5.11.3...r5.11.4) --- updated-dependencies: - dependency-name: org.junit.jupiter:junit-jupiter dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.jupiter:junit-jupiter-engine dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.vintage:junit-vintage-engine dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index a12c61219cfa..ac3a9dc5c147 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -63,7 +63,7 @@ jakarta-servlet-api = "6.1.0" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "11.0.24" -junit = "5.11.3" +junit = "5.11.4" junit-platform = "1.11.3" kafka = "3.9.0" kryo-shaded = "4.0.3" From e4d9c1d7a4edb30ccb7bde5e0278210102d3b628 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 22:14:47 +0100 Subject: [PATCH 289/313] Build: Bump org.assertj:assertj-core from 3.26.3 to 3.27.0 (#11847) Bumps [org.assertj:assertj-core](https://github.com/assertj/assertj) from 3.26.3 to 3.27.0. - [Release notes](https://github.com/assertj/assertj/releases) - [Commits](https://github.com/assertj/assertj/compare/assertj-build-3.26.3...assertj-build-3.27.0) --- updated-dependencies: - dependency-name: org.assertj:assertj-core dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index ac3a9dc5c147..7022d29166d5 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -27,7 +27,7 @@ aircompressor = "0.27" apiguardian = "1.1.2" arrow = "15.0.2" avro = "1.12.0" -assertj-core = "3.26.3" +assertj-core = "3.27.0" awaitility = "4.2.2" awssdk-bom = "2.29.39" azuresdk-bom = "1.2.30" From 12d7ee5c0d6c27ad59bc87b5a12ee2ed3a90f431 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 23:43:54 +0100 Subject: [PATCH 290/313] Build: Bump nessie from 0.101.2 to 0.101.3 (#11852) Bumps `nessie` from 0.101.2 to 0.101.3. Updates `org.projectnessie.nessie:nessie-client` from 0.101.2 to 0.101.3 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.101.2 to 0.101.3 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.101.2 to 0.101.3 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.101.2 to 0.101.3 --- updated-dependencies: - dependency-name: org.projectnessie.nessie:nessie-client dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-jaxrs-testextension dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-testextension dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 7022d29166d5..f562bdbd5df7 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -70,7 +70,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.101.2" +nessie = "0.101.3" netty-buffer = "4.1.116.Final" netty-buffer-compat = "4.1.116.Final" object-client-bundle = "3.3.2" From f7748f20a0f2524ff16c23b665e87838d2a56cc5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 22 Dec 2024 23:44:09 +0100 Subject: [PATCH 291/313] Build: Bump junit-platform from 1.11.3 to 1.11.4 (#11848) Bumps `junit-platform` from 1.11.3 to 1.11.4. Updates `org.junit.platform:junit-platform-suite-api` from 1.11.3 to 1.11.4 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/commits) Updates `org.junit.platform:junit-platform-suite-engine` from 1.11.3 to 1.11.4 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/commits) --- updated-dependencies: - dependency-name: org.junit.platform:junit-platform-suite-api dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.platform:junit-platform-suite-engine dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index f562bdbd5df7..4cf788af945a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -64,7 +64,7 @@ jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "11.0.24" junit = "5.11.4" -junit-platform = "1.11.3" +junit-platform = "1.11.4" kafka = "3.9.0" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" From 55f10ca7cd460cb9ce4a34ef72e69ed2e843b935 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Mon, 23 Dec 2024 10:46:01 +0800 Subject: [PATCH 292/313] Doc: Add status page for different implementations. (#11772) * Partial * Initial * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Update site/docs/status.md Co-authored-by: Fokko Driesprong * Resolve comments * Fix comments * Fix comments * Move position * Remove 's' in implementations * Address comments * Add avro data file formats * Add puffin statistics --------- Co-authored-by: Fokko Driesprong --- site/docs/status.md | 369 ++++++++++++++++++++++++++++++++++++++++++++ site/nav.yml | 3 +- 2 files changed, 371 insertions(+), 1 deletion(-) create mode 100644 site/docs/status.md diff --git a/site/docs/status.md b/site/docs/status.md new file mode 100644 index 000000000000..27968659f91d --- /dev/null +++ b/site/docs/status.md @@ -0,0 +1,369 @@ +--- +title: "Implementation Status" +--- + + +# Implementation Status + +Apache Iceberg's table specification is implemented in multiple languages. This page provides an overview of the current +capabilities. + +## Libraries + +This section lists the libraries that implement the Apache Iceberg specification. + +- [Java](https://mvnrepository.com/artifact/org.apache.iceberg) +- [PyIceberg](https://pypi.org/project/pyiceberg/) +- [Rust](https://crates.io/crates/iceberg) +- [Go](https://pkg.go.dev/github.com/apache/iceberg-go) + +## Data Types + +| Data Type | Java | PyIceberg | Rust | Go | +|----------------|------|-----------|------|----| +| boolean | Y | Y | Y | Y | +| int | Y | Y | Y | Y | +| long | Y | Y | Y | Y | +| float | Y | Y | Y | Y | +| double | Y | Y | Y | Y | +| decimal | Y | Y | Y | Y | +| date | Y | Y | Y | Y | +| time | Y | Y | Y | Y | +| timestamp | Y | Y | Y | Y | +| timestamptz | Y | Y | Y | Y | +| timestamp_ns | Y | Y | Y | Y | +| timestamptz_ns | Y | Y | Y | Y | +| string | Y | Y | Y | Y | +| uuid | Y | Y | Y | Y | +| fixed | Y | Y | Y | Y | +| binary | Y | Y | Y | Y | +| variant | Y | Y | Y | Y | +| list | Y | Y | Y | Y | +| map | Y | Y | Y | Y | +| struct | Y | Y | Y | Y | + +## Data File Formats + +| Format | Java | PyIceberg | Rust | Go | +|---------|------|-----------|------|----| +| Parquet | Y | Y | Y | Y | +| ORC | Y | N | N | N | +| Puffin | Y | N | N | N | +| Avro | Y | N | N | N | + +## File IO + +| Storage | Java | PyIceberg | Rust | Go | +|-------------------|------|-----------|------|----| +| Local Filesystem | Y | Y | Y | Y | +| Hadoop Filesystem | Y | Y | Y | Y | +| S3 Compatible | Y | Y | Y | Y | +| GCS Compatible | Y | Y | Y | Y | + +## Table Maintenance Operations + +### Table Spec V1 + +| Operation | Java | PyIceberg | Rust | Go | +|-----------------------------|------|-----------|------|----| +| Update schema | Y | N | Y | N | +| Update partition spec | Y | N | Y | N | +| Update table properties | Y | Y | Y | N | +| Replace sort order | Y | N | N | N | +| Update table location | Y | N | N | N | +| Update statistics | Y | N | N | N | +| Update partition statistics | Y | N | N | N | +| Expire snapshots | Y | N | N | N | +| Manage snapshots | Y | N | N | N | + +### Table Spec V2 + +| Operation | Java | PyIceberg | Rust | Go | +|-----------------------------|------|-----------|------|----| +| Update schema | Y | Y | N | N | +| Update partition spec | Y | Y | N | N | +| Update table properties | Y | Y | Y | N | +| Replace sort order | Y | N | N | N | +| Update table location | Y | N | N | N | +| Update statistics | Y | N | N | N | +| Update partition statistics | Y | N | N | N | +| Expire snapshots | Y | N | N | N | +| Manage snapshots | Y | N | N | N | + +## Table Update Operations + +### Table Spec V1 + +| Operation | Java | PyIceberg | Rust | Go | +|-------------------|------|-----------|------|----| +| Append data files | Y | Y | N | N | +| Rewrite files | Y | Y | N | N | +| Rewrite manifests | Y | Y | N | N | +| Overwrite files | Y | Y | N | N | +| Delete files | Y | N | N | N | + +### Table Spec V2 + +| Operation | Java | PyIceberg | Rust | Go | +|-------------------|------|-----------|------|----| +| Append data files | Y | Y | N | N | +| Rewrite files | Y | Y | N | N | +| Rewrite manifests | Y | Y | N | N | +| Overwrite files | Y | Y | N | N | +| Row delta | Y | N | N | N | +| Delete files | Y | Y | N | N | + +## Table Read Operations + +### Table Spec V1 + +| Operation | Java | PyIceberg | Rust | Go | +|-----------------------------|------|-----------|------|----| +| Plan with data file | Y | Y | Y | Y | +| Plan with puffin statistics | Y | Y | Y | Y | +| Read data file | Y | N | Y | N | + +### Table Spec V2 + +| Operation | Java | PyIceberg | Rust | Go | +|-----------------------------|------|-----------|------|----| +| Plan with data file | Y | Y | Y | Y | +| Plan with position deletes | Y | Y | N | N | +| Plan with equality deletes | Y | Y | N | N | +| Plan with puffin statistics | Y | N | N | N | +| Read data file | Y | Y | Y | N | +| Read with position deletes | Y | Y | N | N | +| Read with equality deletes | Y | N | N | N | + +## Table Write Operations + +### Table Spec V1 + +| Operation | Java | PyIceberg | Rust | Go | +|-------------|------|-----------|------|----| +| Append data | Y | Y | Y | N | + +### Table Spec V2 + +| Operation | Java | PyIceberg | Rust | Go | +|------------------------|------|-----------|------|----| +| Append data | Y | Y | Y | N | +| Write position deletes | Y | N | N | N | +| Write equality deletes | Y | N | N | N | + +## Catalogs + +### Rest Catalog + +#### Table Spec V1 + +| Table Operation | Java | PyIceberg | Rust | Go | +|-----------------|------|-----------|------|----| +| listTable | Y | Y | Y | Y | +| createTable | Y | Y | Y | Y | +| dropTable | Y | Y | Y | Y | +| loadTable | Y | Y | Y | Y | +| updateTable | Y | Y | Y | Y | +| renameTable | Y | Y | Y | Y | +| tableExists | Y | Y | Y | Y | + +#### Table Spec V2 + +| Table Operation | Java | PyIceberg | Rust | Go | +|-----------------|------|-----------|------|----| +| listTable | Y | Y | Y | Y | +| createTable | Y | Y | Y | Y | +| dropTable | Y | Y | Y | Y | +| loadTable | Y | Y | Y | Y | +| updateTable | Y | Y | Y | Y | +| renameTable | Y | Y | Y | Y | +| tableExists | Y | Y | Y | Y | + +#### View Spec V1 + +| View Operation | Java | PyIceberg | Rust | Go | +|----------------|------|-----------|------|----| +| createView | Y | N | N | N | +| dropView | Y | N | N | N | +| listView | Y | N | N | N | +| viewExists | Y | N | N | N | +| replaceView | Y | N | N | N | +| renameView | Y | N | N | N | + +#### Namespace Operations + +| Namespace Operation | Java | PyIceberg | Rust | Go | +|---------------------------|------|-----------|------|----| +| listNamespaces | Y | Y | Y | Y | +| createNamespace | Y | Y | Y | Y | +| dropNamespace | Y | Y | Y | Y | +| namespaceExists | Y | N | Y | Y | +| updateNamespaceProperties | Y | Y | Y | Y | +| loadNamespaceMetadata | Y | N | Y | Y | + +### Sql Catalog + +The sql catalog is a catalog backed by a sql database, which is called jdbc catalog in java. + +| Database | Java | PyIceberg | Rust | Go | +|----------|------|-----------|------|----| +| Postgres | Y | Y | Y | N | +| MySQL | Y | Y | Y | N | +| SQLite | Y | Y | Y | N | + +#### Table Spec V1 + +| Table Operation | Java | PyIceberg | Rust | Go | +|-----------------|------|-----------|------|----| +| listTable | Y | Y | Y | Y | +| createTable | Y | Y | Y | Y | +| dropTable | Y | Y | Y | Y | +| loadTable | Y | Y | Y | Y | +| updateTable | Y | Y | Y | Y | +| renameTable | Y | Y | Y | Y | +| tableExists | Y | Y | Y | Y | + +#### Table Spec V2 + +| Table Operation | Java | PyIceberg | Rust | Go | +|-----------------|------|-----------|------|----| +| listTable | Y | Y | Y | Y | +| createTable | Y | Y | Y | Y | +| dropTable | Y | Y | Y | Y | +| loadTable | Y | Y | Y | Y | +| updateTable | Y | Y | Y | Y | +| renameTable | Y | Y | Y | Y | +| tableExists | Y | Y | Y | Y | + +#### View Spec V1 + +| View Operation | Java | PyIceberg | Rust | Go | +|----------------|------|-----------|------|----| +| createView | Y | N | N | N | +| dropView | Y | N | N | N | +| listView | Y | N | N | N | +| viewExists | Y | N | N | N | +| replaceView | Y | N | N | N | +| renameView | Y | N | N | N | + +#### Namespace Operations + +| Namespace Operation | Java | PyIceberg | Rust | Go | +|---------------------------|------|-----------|------|----| +| listNamespaces | Y | N | N | N | +| createNamespace | Y | N | N | N | +| dropNamespace | Y | Y | Y | Y | +| namespaceExists | Y | N | N | N | +| updateNamespaceProperties | Y | Y | Y | Y | +| loadNamespaceMetadata | Y | N | N | N | + +### Glue Catalog + +#### Table Spec V1 + +| Table Operation | Java | PyIceberg | Rust | Go | +|-----------------|------|-----------|------|----| +| listTable | Y | Y | Y | Y | +| createTable | Y | Y | Y | Y | +| dropTable | Y | Y | Y | Y | +| loadTable | Y | Y | Y | Y | +| updateTable | Y | Y | Y | Y | +| renameTable | Y | Y | Y | Y | +| tableExists | Y | Y | Y | Y | + +#### Table Spec V2 + +| Table Operation | Java | PyIceberg | Rust | Go | +|-----------------|------|-----------|------|----| +| listTable | Y | Y | Y | Y | +| createTable | Y | Y | Y | Y | +| dropTable | Y | Y | Y | Y | +| loadTable | Y | Y | Y | Y | +| updateTable | Y | Y | Y | Y | +| renameTable | Y | Y | Y | Y | +| tableExists | Y | Y | Y | Y | + +#### View Spec V1 + +| View Operation | Java | PyIceberg | Rust | Go | +|----------------|------|-----------|------|----| +| createView | Y | N | N | N | +| dropView | Y | N | N | N | +| listView | Y | N | N | N | +| viewExists | Y | N | N | N | +| replaceView | Y | N | N | N | +| renameView | Y | N | N | N | + +#### Namespace Operations + +| Namespace Operation | Java | PyIceberg | Rust | Go | +|---------------------------|------|-----------|------|----| +| listNamespaces | Y | N | N | N | +| createNamespace | Y | N | N | N | +| dropNamespace | Y | N | N | N | +| namespaceExists | Y | N | N | N | +| updateNamespaceProperties | Y | Y | Y | Y | +| loadNamespaceMetadata | Y | N | N | N | + +### Hive Metastore Catalog + +#### Table Spec V1 + +| Table Operation | Java | PyIceberg | Rust | Go | +|-----------------|------|-----------|------|----| +| listTable | Y | Y | Y | Y | +| createTable | Y | Y | Y | Y | +| dropTable | Y | Y | Y | Y | +| loadTable | Y | Y | Y | Y | +| updateTable | Y | Y | Y | Y | +| renameTable | Y | Y | Y | Y | +| tableExists | Y | Y | Y | Y | + +#### Table Spec V2 + +| Table Operation | Java | PyIceberg | Rust | Go | +|-----------------|------|-----------|------|----| +| listTable | Y | Y | Y | Y | +| createTable | Y | Y | Y | Y | +| dropTable | Y | Y | Y | Y | +| loadTable | Y | Y | Y | Y | +| updateTable | Y | Y | Y | Y | +| renameTable | Y | Y | Y | Y | +| tableExists | Y | Y | Y | Y | + +#### View Spec V1 + +| View Operation | Java | PyIceberg | Rust | Go | +|----------------|------|-----------|------|----| +| createView | Y | N | N | N | +| dropView | Y | N | N | N | +| listView | Y | N | N | N | +| viewExists | Y | N | N | N | +| replaceView | Y | N | N | N | +| renameView | Y | N | N | N | + +#### Namespace Operations + +| Namespace Operation | Java | PyIceberg | Rust | Go | +|---------------------------|------|-----------|------|----| +| listNamespaces | Y | N | N | N | +| createNamespace | Y | N | N | N | +| dropNamespace | Y | N | N | N | +| namespaceExists | Y | N | N | N | +| updateNamespaceProperties | Y | Y | Y | Y | +| loadNamespaceMetadata | Y | N | N | N | \ No newline at end of file diff --git a/site/nav.yml b/site/nav.yml index da6a56bc6de2..39fffb47692c 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -47,10 +47,11 @@ nav: - View spec: view-spec.md - Puffin spec: puffin-spec.md - AES GCM Stream spec: gcm-stream-spec.md + - Implementation status: status.md - Multi-engine support: multi-engine-support.md - How to release: how-to-release.md - Terms: terms.md - - ASF: + - ASF: - Sponsorship: https://www.apache.org/foundation/thanks.html - Events: https://www.apache.org/events/current-event.html - Privacy: https://privacy.apache.org/policies/privacy-policy-public.html From ca3db931b0f024f0412084751ac85dd4ef2da7e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Mon, 23 Dec 2024 08:41:39 +0100 Subject: [PATCH 293/313] Upgrade to Gradle 8.12 (#11861) --- gradle/wrapper/gradle-wrapper.properties | 4 ++-- gradlew | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index eb1a55be0e15..e1b837a19c22 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,7 +1,7 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionSha256Sum=f397b287023acdba1e9f6fc5ea72d22dd63669d59ed4a289a29b1a76eee151c6 -distributionUrl=https\://services.gradle.org/distributions/gradle-8.11.1-bin.zip +distributionSha256Sum=7a00d51fb93147819aab76024feece20b6b84e420694101f276be952e08bef03 +distributionUrl=https\://services.gradle.org/distributions/gradle-8.12-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 4f63945674c2..4695204cf350 100755 --- a/gradlew +++ b/gradlew @@ -87,7 +87,7 @@ APP_BASE_NAME=${0##*/} APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit if [ ! -e $APP_HOME/gradle/wrapper/gradle-wrapper.jar ]; then - curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.11.1/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.12.0/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. From dbd7d1c6c32834a8708211f19ad6f6901de5433e Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 23 Dec 2024 23:26:32 +0800 Subject: [PATCH 294/313] Build: Fix ignoring `.asf.yaml` in PR (#11860) --- .github/workflows/delta-conversion-ci.yml | 2 +- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 2 +- .github/workflows/java-ci.yml | 2 +- .github/workflows/kafka-connect-ci.yml | 2 +- .github/workflows/spark-ci.yml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index 521d061f6552..e7d6bdd4a5be 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -45,7 +45,7 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'mr/**' - 'hive3/**' diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 22f4f008a215..c355f101553b 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -45,7 +45,7 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'mr/**' - 'hive3/**' diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index d95ca1bd5c6a..2e4ec700a496 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -45,7 +45,7 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'arrow/**' - 'spark/**' diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 1da7a673a865..2f20b5a7760f 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -45,7 +45,7 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'docs/**' - 'site/**' diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index 60cd9188b61d..ac09514218e8 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -45,7 +45,7 @@ on: - '.github/workflows/spark-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'mr/**' - 'flink/**' diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 0d7bd2d3d3e7..6f2efba0d8f1 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -45,7 +45,7 @@ on: - '.github/workflows/site-ci.yml' - '.github/workflows/stale.yml' - '.gitignore' - - '.asf.yml' + - '.asf.yaml' - 'dev/**' - 'site/**' - 'mr/**' From c6d9e0cdd0f30d550bacdcd51acb4e68b9d5b791 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 24 Dec 2024 11:05:07 +0100 Subject: [PATCH 295/313] Gradle: Update `gradlew` with better `APP_HOME` definition (#11869) --- gradlew | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradlew b/gradlew index 4695204cf350..ce1f70c1d22d 100755 --- a/gradlew +++ b/gradlew @@ -84,7 +84,7 @@ done # shellcheck disable=SC2034 APP_BASE_NAME=${0##*/} # Discard cd standard output in case $CDPATH is set (https://github.com/gradle/gradle/issues/25036) -APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit +APP_HOME=$( cd -P "${APP_HOME:-./}" > /dev/null && printf '%s\n' "$PWD" ) || exit if [ ! -e $APP_HOME/gradle/wrapper/gradle-wrapper.jar ]; then curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.12.0/gradle/wrapper/gradle-wrapper.jar From d6d3cf58a99b99984a36802fcec078494ca4b46f Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Tue, 24 Dec 2024 19:05:55 +0900 Subject: [PATCH 296/313] Core, Spark: Avoid deprecated methods in Guava Files (#11865) --- .../org/apache/iceberg/ManifestWriteBenchmark.java | 4 ++-- .../apache/iceberg/hadoop/HadoopTableTestBase.java | 5 +++-- .../spark/action/DeleteOrphanFilesBenchmark.java | 13 +++++++++++-- .../action/IcebergSortCompactionBenchmark.java | 11 ++++++++--- 4 files changed, 24 insertions(+), 9 deletions(-) diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java index c175248241e9..349c8e5d4fa2 100644 --- a/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java @@ -29,7 +29,6 @@ import org.apache.commons.io.FileUtils; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.io.Files; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -96,7 +95,8 @@ public int getFormatVersion() { @Benchmark @Threads(1) public void writeManifestFile(BenchmarkState state) throws IOException { - this.baseDir = Files.createTempDir().getAbsolutePath(); + this.baseDir = + java.nio.file.Files.createTempDirectory("benchmark-").toAbsolutePath().toString(); this.manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); try (ManifestListWriter listWriter = diff --git a/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java index b8f7dcb80ebc..76d67938b100 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java @@ -152,13 +152,14 @@ TableMetadata readMetadataVersion(int version) { } int readVersionHint() throws IOException { - return Integer.parseInt(Files.readFirstLine(versionHintFile, StandardCharsets.UTF_8)); + return Integer.parseInt( + Files.asCharSource(versionHintFile, StandardCharsets.UTF_8).readFirstLine()); } void replaceVersionHint(int version) throws IOException { // remove the checksum that will no longer match new File(metadataDir, ".version-hint.text.crc").delete(); - Files.write(String.valueOf(version), versionHintFile, StandardCharsets.UTF_8); + Files.asCharSink(versionHintFile, StandardCharsets.UTF_8).write(String.valueOf(version)); } /* diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java index fd0dcac6bbcc..64edb1002e99 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java @@ -20,6 +20,9 @@ import static org.apache.spark.sql.functions.lit; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; import java.sql.Timestamp; import java.util.List; import java.util.Locale; @@ -32,7 +35,6 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.DeleteOrphanFiles; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.io.Files; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSessionCatalog; import org.apache.iceberg.spark.actions.SparkActions; @@ -161,7 +163,14 @@ private Table table() { } private String catalogWarehouse() { - return Files.createTempDir().getAbsolutePath() + "/" + UUID.randomUUID() + "/"; + try { + return Files.createTempDirectory("benchmark-").toAbsolutePath() + + "/" + + UUID.randomUUID() + + "/"; + } catch (IOException e) { + throw new UncheckedIOException(e); + } } private void setupSpark() { diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java index 95bebc7caed4..a5d0456b0b28 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java @@ -26,6 +26,7 @@ import static org.apache.spark.sql.functions.expr; import java.io.IOException; +import java.nio.file.Files; import java.util.Collections; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -36,7 +37,6 @@ import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.actions.SizeBasedFileRewriter; -import org.apache.iceberg.relocated.com.google.common.io.Files; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkSessionCatalog; @@ -372,8 +372,13 @@ protected final SparkSession spark() { } protected String getCatalogWarehouse() { - String location = Files.createTempDir().getAbsolutePath() + "/" + UUID.randomUUID() + "/"; - return location; + try { + String location = + Files.createTempDirectory("benchmark-").toAbsolutePath() + "/" + UUID.randomUUID() + "/"; + return location; + } catch (IOException e) { + throw new RuntimeException(e); + } } protected void cleanupFiles() throws IOException { From 1b5886d0e6a97043259b8cda18d146feb5a552a4 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Wed, 25 Dec 2024 00:24:22 +0900 Subject: [PATCH 297/313] Core: Don't clear snapshotLog in `TableMetadata.removeRef` (#11779) --- .../org/apache/iceberg/TableMetadata.java | 1 - .../org/apache/iceberg/TestTableMetadata.java | 18 ++++++++++ .../apache/iceberg/catalog/CatalogTests.java | 34 +++++++++++++++++++ 3 files changed, 52 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 9f6ffbcc8714..19afb7af04aa 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -1286,7 +1286,6 @@ public Builder setRef(String name, SnapshotRef ref) { public Builder removeRef(String name) { if (SnapshotRef.MAIN_BRANCH.equals(name)) { this.currentSnapshotId = -1; - snapshotLog.clear(); } SnapshotRef ref = refs.remove(name); diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 6d066e8a654c..45aa211e5187 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -1693,6 +1693,24 @@ public void buildReplacementKeepsSnapshotLog() throws Exception { .containsExactlyElementsOf(metadata.snapshotLog()); } + @Test + public void removeRefKeepsSnapshotLog() throws Exception { + TableMetadata metadata = + TableMetadataParser.fromJson(readTableMetadataInputFile("TableMetadataV2Valid.json")); + assertThat(metadata.currentSnapshot()).isNotNull(); + assertThat(metadata.snapshots()).hasSize(2); + assertThat(metadata.snapshotLog()).hasSize(2); + + TableMetadata removeRef = + TableMetadata.buildFrom(metadata).removeRef(SnapshotRef.MAIN_BRANCH).build(); + + assertThat(removeRef.currentSnapshot()).isNull(); + assertThat(removeRef.snapshots()).hasSize(2).containsExactlyElementsOf(metadata.snapshots()); + assertThat(removeRef.snapshotLog()) + .hasSize(2) + .containsExactlyElementsOf(metadata.snapshotLog()); + } + @Test public void testConstructV3Metadata() { TableMetadata.newTableMetadata( diff --git a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java index ecdf30463472..b1eef4c015ac 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -41,6 +41,7 @@ import org.apache.iceberg.FileScanTask; import org.apache.iceberg.FilesTable; import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReachableFileUtil; import org.apache.iceberg.ReplaceSortOrder; @@ -2151,6 +2152,39 @@ public void testReplaceTransactionRequiresTableExists() { .hasMessageStartingWith("Table does not exist: newdb.table"); } + @Test + public void testReplaceTableKeepsSnapshotLog() { + C catalog = catalog(); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + + catalog.createTable(TABLE, SCHEMA); + + Table table = catalog.loadTable(TABLE); + table.newAppend().appendFile(FILE_A).commit(); + + List snapshotLogBeforeReplace = + ((BaseTable) table).operations().current().snapshotLog(); + assertThat(snapshotLogBeforeReplace).hasSize(1); + HistoryEntry snapshotBeforeReplace = snapshotLogBeforeReplace.get(0); + + Transaction replaceTableTransaction = catalog.newReplaceTableTransaction(TABLE, SCHEMA, false); + replaceTableTransaction.newAppend().appendFile(FILE_A).commit(); + replaceTableTransaction.commitTransaction(); + table.refresh(); + + List snapshotLogAfterReplace = + ((BaseTable) table).operations().current().snapshotLog(); + HistoryEntry snapshotAfterReplace = snapshotLogAfterReplace.get(1); + + assertThat(snapshotAfterReplace).isNotEqualTo(snapshotBeforeReplace); + assertThat(snapshotLogAfterReplace) + .hasSize(2) + .containsExactly(snapshotBeforeReplace, snapshotAfterReplace); + } + @Test public void testConcurrentReplaceTransactions() { C catalog = catalog(); From 4eb9f7ffa9b0f4b0adee7169588d81f46149af9f Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 26 Dec 2024 04:06:18 +0900 Subject: [PATCH 298/313] Core: Replace deprecated Schema.toString with SchemaFormatter (#11867) --- .../java/org/apache/iceberg/avro/TestSchemaConversions.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java b/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java index e135364bca66..d9dc49d17257 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java @@ -32,6 +32,7 @@ import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; +import org.apache.avro.SchemaFormatter; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type; @@ -320,7 +321,8 @@ public void testComplexSchema() { "properties", Types.MapType.ofOptional(18, 19, Types.StringType.get(), Types.StringType.get()))); - AvroSchemaUtil.convert(schema, "newTableName").toString(true); + SchemaFormatter.getInstance("json/pretty") + .format(AvroSchemaUtil.convert(schema, "newTableName")); } @Test From bb27030a4c857e24ab2c7479c3b950388925dd60 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 26 Dec 2024 05:26:27 +0800 Subject: [PATCH 299/313] Build: Fix ignoring `license-check.yml` in PR (#11873) --- .github/workflows/delta-conversion-ci.yml | 2 +- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 2 +- .github/workflows/java-ci.yml | 2 +- .github/workflows/kafka-connect-ci.yml | 2 +- .github/workflows/spark-ci.yml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index e7d6bdd4a5be..7b8fcfc992b8 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index c355f101553b..6171d24e3c62 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index 2e4ec700a496..0a9357fddae6 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 2f20b5a7760f..87f6fcea2e11 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index ac09514218e8..95c8ee5bfb39 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -37,7 +37,7 @@ on: - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 6f2efba0d8f1..022c61084435 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -38,7 +38,7 @@ on: - '.github/workflows/jmh-benchmarks-ci.yml' - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - - '.github/workflows/licence-check.yml' + - '.github/workflows/license-check.yml' - '.github/workflows/open-api.yml' - '.github/workflows/publish-snapshot.yml' - '.github/workflows/recurring-jmh-benchmarks.yml' From fc3f705ab45a6eefcf2be547c5193173ed42b807 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 26 Dec 2024 16:19:16 +0900 Subject: [PATCH 300/313] API: Replace deprecated `asList` with `asInstanceOf` (#11875) --- .../java/org/apache/iceberg/util/TestExceptionUtil.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java b/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java index e6c3cf5c20ba..20315176b11a 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java +++ b/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.Arrays; +import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.Test; public class TestExceptionUtil { @@ -53,7 +54,7 @@ public void testRunSafely() { .isInstanceOf(CustomCheckedException.class) .isEqualTo(exc) .extracting(e -> Arrays.asList(e.getSuppressed())) - .asList() + .asInstanceOf(InstanceOfAssertFactories.LIST) .hasSize(2) .containsExactly(suppressedOne, suppressedTwo); } @@ -82,7 +83,7 @@ public void testRunSafelyTwoExceptions() { .isInstanceOf(CustomCheckedException.class) .isEqualTo(exc) .extracting(e -> Arrays.asList(e.getSuppressed())) - .asList() + .asInstanceOf(InstanceOfAssertFactories.LIST) .hasSize(2) .containsExactly(suppressedOne, suppressedTwo); } @@ -112,7 +113,7 @@ public void testRunSafelyThreeExceptions() { .isInstanceOf(CustomCheckedException.class) .isEqualTo(exc) .extracting(e -> Arrays.asList(e.getSuppressed())) - .asList() + .asInstanceOf(InstanceOfAssertFactories.LIST) .hasSize(2) .containsExactly(suppressedOne, suppressedTwo); } @@ -137,7 +138,7 @@ public void testRunSafelyRuntimeExceptions() { .isInstanceOf(RuntimeException.class) .isEqualTo(exc) .extracting(e -> Arrays.asList(e.getSuppressed())) - .asList() + .asInstanceOf(InstanceOfAssertFactories.LIST) .hasSize(2) .containsExactly(suppressedOne, suppressedTwo); } From de54a0867306c54655bb101fecb14d7264ed62de Mon Sep 17 00:00:00 2001 From: big face cat <731030576@qq.com> Date: Fri, 27 Dec 2024 09:28:40 +0800 Subject: [PATCH 301/313] Flink: Avoid RANGE mode broken chain when write parallelism changes (#11702) Co-authored-by: huyuanfeng --- .../apache/iceberg/flink/sink/FlinkSink.java | 14 ++++++- .../TestFlinkIcebergSinkDistributionMode.java | 41 +++++++++++-------- 2 files changed, 35 insertions(+), 20 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 18f3557beeff..4acd32e13c76 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -33,6 +33,7 @@ import java.util.Map; import java.util.Set; import java.util.function.Function; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; @@ -667,8 +668,17 @@ private DataStream distributeDataStream( return shuffleStream .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) - .filter(StatisticsOrRecord::hasRecord) - .map(StatisticsOrRecord::record); + .flatMap( + (FlatMapFunction) + (statisticsOrRecord, out) -> { + if (statisticsOrRecord.hasRecord()) { + out.collect(statisticsOrRecord.record()); + } + }) + // Set the parallelism same as writerParallelism to + // promote operator chaining with the downstream writer operator + .setParallelism(writerParallelism) + .returns(RowData.class); default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java index aa9a0291b38f..84fa48e38b70 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -82,13 +82,18 @@ public class TestFlinkIcebergSinkDistributionMode extends TestFlinkIcebergSinkBa @Parameter(index = 1) private boolean partitioned; - @Parameters(name = "parallelism = {0}, partitioned = {1}") + @Parameter(index = 2) + private int writeParallelism; + + @Parameters(name = "parallelism = {0}, partitioned = {1}, writeParallelism = {2}") public static Object[][] parameters() { return new Object[][] { - {1, true}, - {1, false}, - {2, true}, - {2, false} + {1, true, 1}, + {1, false, 1}, + {2, true, 2}, + {2, false, 2}, + {1, true, 2}, + {1, false, 2}, }; } @@ -110,7 +115,7 @@ public void before() throws IOException { MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) - .setMaxParallelism(parallelism); + .setMaxParallelism(Math.max(parallelism, writeParallelism)); this.tableLoader = CATALOG_EXTENSION.tableLoader(); } @@ -180,7 +185,7 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) .table(table) .tableLoader(tableLoader) - .writeParallelism(parallelism) + .writeParallelism(writeParallelism) .setAll(newProps); assertThatThrownBy(builder::append) @@ -206,7 +211,7 @@ public void testRangeDistributionWithoutSortOrderUnpartitioned() throws Exceptio FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) .table(table) .tableLoader(tableLoader) - .writeParallelism(parallelism); + .writeParallelism(writeParallelism); // Range distribution requires either sort order or partition spec defined assertThatThrownBy(builder::append) @@ -233,7 +238,7 @@ public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) .table(table) .tableLoader(tableLoader) - .writeParallelism(parallelism); + .writeParallelism(writeParallelism); // sort based on partition columns builder.append(); @@ -307,7 +312,7 @@ public void testRangeDistributionWithSortOrder() throws Exception { FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) .table(table) .tableLoader(tableLoader) - .writeParallelism(parallelism) + .writeParallelism(writeParallelism) .rangeDistributionStatisticsType(StatisticsType.Map) .append(); env.execute(getClass().getSimpleName()); @@ -343,9 +348,9 @@ public void testRangeDistributionWithSortOrder() throws Exception { List addedDataFiles = Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); // each writer task should only write one file for non-partition sort column - assertThat(addedDataFiles).hasSize(parallelism); + assertThat(addedDataFiles).hasSize(writeParallelism); // verify there is no overlap in min-max stats range - if (parallelism == 2) { + if (parallelism > 1) { assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); } } @@ -368,7 +373,7 @@ public void testRangeDistributionSketchWithSortOrder() throws Exception { FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) .table(table) .tableLoader(tableLoader) - .writeParallelism(parallelism) + .writeParallelism(writeParallelism) .rangeDistributionStatisticsType(StatisticsType.Sketch) .append(); env.execute(getClass().getSimpleName()); @@ -399,9 +404,9 @@ public void testRangeDistributionSketchWithSortOrder() throws Exception { List addedDataFiles = Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); // each writer task should only write one file for non-partition sort column - assertThat(addedDataFiles).hasSize(parallelism); + assertThat(addedDataFiles).hasSize(writeParallelism); // verify there is no overlap in min-max stats range - if (parallelism == 2) { + if (writeParallelism > 2) { assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); } } @@ -437,7 +442,7 @@ public void testRangeDistributionStatisticsMigration() throws Exception { FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) .table(table) .tableLoader(tableLoader) - .writeParallelism(parallelism) + .writeParallelism(writeParallelism) .rangeDistributionStatisticsType(StatisticsType.Auto) .append(); env.execute(getClass().getSimpleName()); @@ -469,9 +474,9 @@ public void testRangeDistributionStatisticsMigration() throws Exception { Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); // each writer task should only write one file for non-partition sort column // sometimes - assertThat(addedDataFiles).hasSize(parallelism); + assertThat(addedDataFiles).hasSize(writeParallelism); // verify there is no overlap in min-max stats range - if (parallelism == 2) { + if (writeParallelism > 1) { assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); } } From 607e2fbecb666f134f550548abe5795060bab472 Mon Sep 17 00:00:00 2001 From: Gabriel Igliozzi Date: Sat, 28 Dec 2024 11:19:00 -0500 Subject: [PATCH 302/313] Update `README.md` with `iceberg-cpp` (#11882) --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 5c6e5fd96d35..394d9aea0d3d 100644 --- a/README.md +++ b/README.md @@ -97,3 +97,4 @@ This repository contains the Java implementation of Iceberg. Other implementatio * **Go**: [iceberg-go](https://github.com/apache/iceberg-go) * **PyIceberg** (Python): [iceberg-python](https://github.com/apache/iceberg-python) * **Rust**: [iceberg-rust](https://github.com/apache/iceberg-rust) +* **C++**: [iceberg-cpp](https://github.com/apache/iceberg-cpp) From 0029d6a7e8511ed081e7863ed60617706edd4445 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 29 Dec 2024 21:36:40 +0100 Subject: [PATCH 303/313] Build: Bump software.amazon.awssdk:bom from 2.29.39 to 2.29.43 (#11886) Bumps software.amazon.awssdk:bom from 2.29.39 to 2.29.43. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 4cf788af945a..556187ceecb8 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.27.0" awaitility = "4.2.2" -awssdk-bom = "2.29.39" +awssdk-bom = "2.29.43" azuresdk-bom = "1.2.30" awssdk-s3accessgrants = "2.3.0" caffeine = "2.9.3" From 3fa8a46bca48f20aa9ad90ea66900760069de51b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 29 Dec 2024 21:37:00 +0100 Subject: [PATCH 304/313] Build: Bump mkdocs-awesome-pages-plugin from 2.10.0 to 2.10.1 (#11885) Bumps [mkdocs-awesome-pages-plugin](https://github.com/lukasgeiter/mkdocs-awesome-pages-plugin) from 2.10.0 to 2.10.1. - [Release notes](https://github.com/lukasgeiter/mkdocs-awesome-pages-plugin/releases) - [Commits](https://github.com/lukasgeiter/mkdocs-awesome-pages-plugin/compare/v2.10.0...v2.10.1) --- updated-dependencies: - dependency-name: mkdocs-awesome-pages-plugin dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 4d1be7a4bc80..8bf7cb5c4429 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -mkdocs-awesome-pages-plugin==2.10.0 +mkdocs-awesome-pages-plugin==2.10.1 mkdocs-macros-plugin==1.3.7 mkdocs-material==9.5.49 mkdocs-material-extensions==1.3.1 From 7f14032be8c0538bfa59aba9951ec8a6001035e3 Mon Sep 17 00:00:00 2001 From: Shohei Okumiya Date: Mon, 30 Dec 2024 05:52:21 +0900 Subject: [PATCH 305/313] Core: Fix typo in HadoopTableOperations (#11880) --- .../java/org/apache/iceberg/hadoop/HadoopTableOperations.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java index 1be351ec7370..3ec5e253f88b 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java @@ -344,7 +344,7 @@ int findVersion() { return maxVersion; } catch (IOException io) { - LOG.warn("Error trying to recover version-hint.txt data for {}", versionHintFile, e); + LOG.warn("Error trying to recover the latest version number for {}", versionHintFile, io); return 0; } } From e3f50e5c62d01f3f31239d197ef281fc36cf31fa Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 30 Dec 2024 22:04:36 +0100 Subject: [PATCH 306/313] Revert "Hive: close the fileIO client when closing the hive catalog (#10771)" (#11858) This reverts commit 7e2920af400e5d559f8f1742b1043787b0477d74. --- .../org/apache/iceberg/hive/HiveCatalog.java | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java index 9fd7c6f2eeb0..89e79c8c4722 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.hive; -import java.io.IOException; import java.util.List; import java.util.Map; import java.util.Set; @@ -53,7 +52,6 @@ import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.FileIOTracker; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -92,7 +90,6 @@ public class HiveCatalog extends BaseMetastoreViewCatalog private ClientPool clients; private boolean listAllTables = false; private Map catalogProperties; - private FileIOTracker fileIOTracker; public HiveCatalog() {} @@ -125,7 +122,6 @@ public void initialize(String inputName, Map properties) { : CatalogUtil.loadFileIO(fileIOImpl, properties, conf); this.clients = new CachedClientPool(conf, properties); - this.fileIOTracker = new FileIOTracker(); } @Override @@ -666,10 +662,7 @@ private boolean isValidateNamespace(Namespace namespace) { public TableOperations newTableOps(TableIdentifier tableIdentifier) { String dbName = tableIdentifier.namespace().level(0); String tableName = tableIdentifier.name(); - HiveTableOperations ops = - new HiveTableOperations(conf, clients, fileIO, name, dbName, tableName); - fileIOTracker.track(ops); - return ops; + return new HiveTableOperations(conf, clients, fileIO, name, dbName, tableName); } @Override @@ -798,14 +791,6 @@ protected Map properties() { return catalogProperties == null ? ImmutableMap.of() : catalogProperties; } - @Override - public void close() throws IOException { - super.close(); - if (fileIOTracker != null) { - fileIOTracker.close(); - } - } - @VisibleForTesting void setListAllTables(boolean listAllTables) { this.listAllTables = listAllTables; From ab6365d42d6c3bfa7054ecd98dde8407e5ba1201 Mon Sep 17 00:00:00 2001 From: Shohei Okumiya Date: Fri, 3 Jan 2025 04:38:24 +0900 Subject: [PATCH 307/313] Docs: Add history to Hive's metadata tables (#11902) --- docs/docs/hive.md | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/docs/docs/hive.md b/docs/docs/hive.md index 69c0f6279538..13f013158326 100644 --- a/docs/docs/hive.md +++ b/docs/docs/hive.md @@ -747,18 +747,20 @@ To reference a metadata table the full name of the table should be used, like: Currently the following metadata tables are available in Hive: -* all_data_files -* all_delete_files -* all_entries all_files -* all_manifests -* data_files -* delete_files -* entries -* files -* manifests -* metadata_log_entries -* partitions -* refs +* all_data_files +* all_delete_files +* all_entries +* all_files +* all_manifests +* data_files +* delete_files +* entries +* files +* history +* manifests +* metadata_log_entries +* partitions +* refs * snapshots ```sql From 3b0004343a712e77a6b00b84caaee9dddebfa94d Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Fri, 3 Jan 2025 17:54:38 +0900 Subject: [PATCH 308/313] Doc: Fix format of Hive (#11892) --- docs/docs/hive.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/docs/hive.md b/docs/docs/hive.md index 13f013158326..a3cd79539c43 100644 --- a/docs/docs/hive.md +++ b/docs/docs/hive.md @@ -300,6 +300,7 @@ The result is: | i | BUCKET\[2\] | NULL The supported transformations for Hive are the same as for Spark: + * years(ts): partition by year * months(ts): partition by month * days(ts) or date(ts): equivalent to dateint partitioning @@ -841,8 +842,6 @@ ALTER TABLE ice_t EXECUTE ROLLBACK(1111); ### Compaction Hive 4 supports full table compaction of Iceberg tables using the following commands: -* Using the `ALTER TABLE ... COMPACT` syntax -* Using the `OPTIMIZE TABLE ... REWRITE DATA` syntax ```sql -- Using the ALTER TABLE ... COMPACT syntax ALTER TABLE t COMPACT 'major'; From 4d35682295b6692a194b621e5afb880d9ccfab6f Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Fri, 3 Jan 2025 17:00:23 +0800 Subject: [PATCH 309/313] Flink: Backport #11662 Fix range distribution npe when value is null to Flink 1.18 and 1.19 (#11745) --- .../sink/shuffle/CompletedStatistics.java | 17 +++++ .../CompletedStatisticsSerializer.java | 12 +++ .../shuffle/DataStatisticsCoordinator.java | 3 +- .../flink/sink/shuffle/SortKeySerializer.java | 67 ++++++++++++++--- .../flink/sink/shuffle/StatisticsUtil.java | 25 ++++++- .../TestFlinkIcebergSinkDistributionMode.java | 39 ++++++++++ .../TestCompletedStatisticsSerializer.java | 49 ++++++++++++ .../TestDataStatisticsCoordinator.java | 74 +++++++++++++++++++ .../shuffle/TestDataStatisticsOperator.java | 30 ++++++++ .../TestSortKeySerializerPrimitives.java | 4 +- .../TestSortKeySerializerSnapshot.java | 22 ++++++ .../sink/shuffle/CompletedStatistics.java | 17 +++++ .../CompletedStatisticsSerializer.java | 12 +++ .../shuffle/DataStatisticsCoordinator.java | 3 +- .../flink/sink/shuffle/SortKeySerializer.java | 65 ++++++++++++++-- .../flink/sink/shuffle/StatisticsUtil.java | 25 ++++++- .../TestFlinkIcebergSinkDistributionMode.java | 39 ++++++++++ .../TestCompletedStatisticsSerializer.java | 49 ++++++++++++ .../TestDataStatisticsCoordinator.java | 74 +++++++++++++++++++ .../shuffle/TestDataStatisticsOperator.java | 30 ++++++++ .../TestSortKeySerializerPrimitives.java | 4 +- .../TestSortKeySerializerSnapshot.java | 22 ++++++ 22 files changed, 651 insertions(+), 31 deletions(-) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index e4cba174f0f2..a8bf0f839e49 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -108,4 +108,21 @@ boolean isEmpty() { return keyFrequency().isEmpty(); } } + + boolean isValid() { + if (type == StatisticsType.Sketch) { + if (null == keySamples) { + return false; + } + } else { + if (null == keyFrequency()) { + return false; + } + if (keyFrequency().values().contains(null)) { + return false; + } + } + + return true; + } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java index 7f55188e7f8c..3659ce217c58 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -48,6 +48,18 @@ class CompletedStatisticsSerializer extends TypeSerializer this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); } + public void changeSortKeySerializerVersion(int version) { + if (sortKeySerializer instanceof SortKeySerializer) { + ((SortKeySerializer) sortKeySerializer).setVersion(version); + } + } + + public void changeSortKeySerializerVersionLatest() { + if (sortKeySerializer instanceof SortKeySerializer) { + ((SortKeySerializer) sortKeySerializer).restoreToLatestVersion(); + } + } + @Override public boolean isImmutableType() { return false; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index e2a282efd82e..4f2afd60fed1 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -370,7 +370,8 @@ public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); this.completedStatistics = StatisticsUtil.deserializeCompletedStatistics( - checkpointData, completedStatisticsSerializer); + checkpointData, (CompletedStatisticsSerializer) completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed this.globalStatistics = globalStatistics( diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index 61c6973463ef..9947e2b78e3a 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -53,9 +53,12 @@ class SortKeySerializer extends TypeSerializer { private final int size; private final Types.NestedField[] transformedFields; + private int version; + private transient SortKey sortKey; - SortKeySerializer(Schema schema, SortOrder sortOrder) { + SortKeySerializer(Schema schema, SortOrder sortOrder, int version) { + this.version = version; this.schema = schema; this.sortOrder = sortOrder; this.size = sortOrder.fields().size(); @@ -76,6 +79,10 @@ class SortKeySerializer extends TypeSerializer { } } + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this(schema, sortOrder, SortKeySerializerSnapshot.CURRENT_VERSION); + } + private SortKey lazySortKey() { if (sortKey == null) { this.sortKey = new SortKey(schema, sortOrder); @@ -84,6 +91,18 @@ private SortKey lazySortKey() { return sortKey; } + public int getLatestVersion() { + return snapshotConfiguration().getCurrentVersion(); + } + + public void restoreToLatestVersion() { + this.version = snapshotConfiguration().getCurrentVersion(); + } + + public void setVersion(int version) { + this.version = version; + } + @Override public boolean isImmutableType() { return false; @@ -125,6 +144,16 @@ public void serialize(SortKey record, DataOutputView target) throws IOException for (int i = 0; i < size; ++i) { int fieldId = transformedFields[i].fieldId(); Type.TypeID typeId = transformedFields[i].type().typeId(); + if (version > 1) { + Object value = record.get(i, Object.class); + if (value == null) { + target.writeBoolean(true); + continue; + } else { + target.writeBoolean(false); + } + } + switch (typeId) { case BOOLEAN: target.writeBoolean(record.get(i, Boolean.class)); @@ -193,6 +222,14 @@ public SortKey deserialize(SortKey reuse, DataInputView source) throws IOExcepti reuse.size(), size); for (int i = 0; i < size; ++i) { + if (version > 1) { + boolean isNull = source.readBoolean(); + if (isNull) { + reuse.set(i, null); + continue; + } + } + int fieldId = transformedFields[i].fieldId(); Type.TypeID typeId = transformedFields[i].type().typeId(); switch (typeId) { @@ -277,11 +314,13 @@ public TypeSerializerSnapshot snapshotConfiguration() { } public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { - private static final int CURRENT_VERSION = 1; + private static final int CURRENT_VERSION = 2; private Schema schema; private SortOrder sortOrder; + private int version = CURRENT_VERSION; + /** Constructor for read instantiation. */ @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public SortKeySerializerSnapshot() { @@ -311,10 +350,16 @@ public void writeSnapshot(DataOutputView out) throws IOException { @Override public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { - if (readVersion == 1) { - readV1(in); - } else { - throw new IllegalArgumentException("Unknown read version: " + readVersion); + switch (readVersion) { + case 1: + read(in); + this.version = 1; + break; + case 2: + read(in); + break; + default: + throw new IllegalArgumentException("Unknown read version: " + readVersion); } } @@ -325,9 +370,13 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } - // Sort order should be identical SortKeySerializerSnapshot newSnapshot = (SortKeySerializerSnapshot) newSerializer.snapshotConfiguration(); + if (newSnapshot.getCurrentVersion() == 1 && this.getCurrentVersion() == 2) { + return TypeSerializerSchemaCompatibility.compatibleAfterMigration(); + } + + // Sort order should be identical if (!sortOrder.sameOrder(newSnapshot.sortOrder)) { return TypeSerializerSchemaCompatibility.incompatible(); } @@ -351,10 +400,10 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( public TypeSerializer restoreSerializer() { Preconditions.checkState(schema != null, "Invalid schema: null"); Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); - return new SortKeySerializer(schema, sortOrder); + return new SortKeySerializer(schema, sortOrder, version); } - private void readV1(DataInputView in) throws IOException { + private void read(DataInputView in) throws IOException { String schemaJson = StringUtils.readString(in); String sortOrderJson = StringUtils.readString(in); this.schema = SchemaParser.fromJson(schemaJson); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java index 5d48ec57ca49..f2efc7fa9834 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java @@ -73,12 +73,29 @@ static byte[] serializeCompletedStatistics( } static CompletedStatistics deserializeCompletedStatistics( - byte[] bytes, TypeSerializer statisticsSerializer) { + byte[] bytes, CompletedStatisticsSerializer statisticsSerializer) { try { DataInputDeserializer input = new DataInputDeserializer(bytes); - return statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + CompletedStatistics completedStatistics = statisticsSerializer.deserialize(input); + if (!completedStatistics.isValid()) { + throw new RuntimeException("Fail to deserialize aggregated statistics,change to v1"); + } + + return completedStatistics; + } catch (Exception e) { + try { + // If we restore from a lower version, the new version of SortKeySerializer cannot correctly + // parse the checkpointData, so we need to first switch the version to v1. Once the state + // data is successfully parsed, we need to switch the serialization version to the latest + // version to parse the subsequent data passed from the TM. + statisticsSerializer.changeSortKeySerializerVersion(1); + DataInputDeserializer input = new DataInputDeserializer(bytes); + CompletedStatistics deserialize = statisticsSerializer.deserialize(input); + statisticsSerializer.changeSortKeySerializerVersionLatest(); + return deserialize; + } catch (IOException ioException) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", ioException); + } } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java index df8c3c79d3e3..aa9a0291b38f 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -46,6 +46,7 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; 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.Maps; @@ -252,6 +253,44 @@ public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); } + @TestTemplate + public void testRangeDistributionWithNullValue() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + List> charRows = createCharRows(numOfCheckpoints, 10); + charRows.add(ImmutableList.of(Row.of(1, null))); + DataStream dataStream = + env.addSource(createRangeDistributionBoundedSource(charRows), ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // sort based on partition columns + builder.append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + } + @TestTemplate public void testRangeDistributionWithSortOrder() throws Exception { table diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java index 4ee9888934a8..1975d7e8d654 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -19,11 +19,15 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; public class TestCompletedStatisticsSerializer extends SerializerTestBase { @@ -51,4 +55,49 @@ protected CompletedStatistics[] getTestData() { CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) }; } + + @Test + public void testSerializer() throws Exception { + TypeSerializer completedStatisticsTypeSerializer = createSerializer(); + CompletedStatistics[] data = getTestData(); + DataOutputSerializer output = new DataOutputSerializer(1024); + completedStatisticsTypeSerializer.serialize(data[0], output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + CompletedStatistics deserialized = completedStatisticsTypeSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(data[0]); + } + + @Test + public void testRestoreOldVersionSerializer() throws Exception { + CompletedStatisticsSerializer completedStatisticsTypeSerializer = + (CompletedStatisticsSerializer) createSerializer(); + completedStatisticsTypeSerializer.changeSortKeySerializerVersion(1); + CompletedStatistics[] data = getTestData(); + DataOutputSerializer output = new DataOutputSerializer(1024); + completedStatisticsTypeSerializer.serialize(data[0], output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + completedStatisticsTypeSerializer.changeSortKeySerializerVersionLatest(); + CompletedStatistics completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + serializedBytes, completedStatisticsTypeSerializer); + assertThat(completedStatistics).isEqualTo(data[0]); + } + + @Test + public void testRestoreNewSerializer() throws Exception { + CompletedStatisticsSerializer completedStatisticsTypeSerializer = + (CompletedStatisticsSerializer) createSerializer(); + CompletedStatistics[] data = getTestData(); + DataOutputSerializer output = new DataOutputSerializer(1024); + completedStatisticsTypeSerializer.serialize(data[0], output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + CompletedStatistics completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + serializedBytes, completedStatisticsTypeSerializer); + assertThat(completedStatistics).isEqualTo(data[0]); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index a08a8a73e80c..acecc5b60af1 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -152,6 +152,80 @@ public void testDataStatisticsEventHandling(StatisticsType type) throws Exceptio } } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testDataStatisticsEventHandlingWithNullValue(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + SortKey nullSortKey = Fixtures.SORT_KEY.copy(); + nullSortKey.set(0, null); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + nullSortKey, + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + nullSortKey, + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + Map keyFrequency = + ImmutableMap.of(nullSortKey, 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 5L); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(NUM_SUBTASKS, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(keyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + nullSortKey, + nullSortKey, + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.checkpointId()).isEqualTo(1L); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + } + } + } + @Test public void testRequestGlobalStatisticsEventHandling() throws Exception { try (DataStatisticsCoordinator dataStatisticsCoordinator = diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index c760f1ba96d3..70837f5ef480 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -34,6 +34,8 @@ import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; @@ -135,6 +137,34 @@ public void testProcessElement(StatisticsType type) throws Exception { } } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testProcessElementWithNull(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + operator.processElement(new StreamRecord<>(GenericRowData.of(null, 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + + DataStatistics localStatistics = operator.localStatistics(); + SortKeySerializer sortKeySerializer = + new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); + DataStatisticsSerializer taskStatisticsSerializer = + new DataStatisticsSerializer(sortKeySerializer); + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + taskStatisticsSerializer.serialize(localStatistics, outputView); + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + DataStatistics dataStatistics = taskStatisticsSerializer.deserialize(inputView); + + testHarness.endInput(); + + assertThat(localStatistics).isEqualTo(dataStatistics); + } + } + @ParameterizedTest @EnumSource(StatisticsType.class) public void testOperatorOutput(StatisticsType type) throws Exception { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java index 54cceae6e55b..ac2e2784e681 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -80,8 +80,8 @@ public void testSerializationSize() throws Exception { byte[] serializedBytes = output.getCopyOfBuffer(); assertThat(serializedBytes.length) .as( - "Serialized bytes for sort key should be 38 bytes (34 UUID text + 4 byte integer of string length") - .isEqualTo(38); + "Serialized bytes for sort key should be 39 bytes (34 UUID text + 4 byte integer of string length + 1 byte of isnull flag") + .isEqualTo(39); DataInputDeserializer input = new DataInputDeserializer(serializedBytes); SortKey deserialized = serializer.deserialize(input); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java index 012654603b04..2359f4b32c35 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -73,6 +73,28 @@ public void testRestoredSerializer() throws Exception { assertThat(deserialized).isEqualTo(sortKey); } + @Test + public void testRestoredOldSerializer() throws Exception { + RowData rowData = GenericRowData.of(StringData.fromString("str"), 1); + RowDataWrapper rowDataWrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(struct); + + SortKeySerializer originalSerializer = new SortKeySerializer(SCHEMA, SORT_ORDER, 1); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + ((SortKeySerializer) restoredSerializer).setVersion(1); + DataOutputSerializer output = new DataOutputSerializer(1024); + originalSerializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = restoredSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } + @Test public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index e4cba174f0f2..a8bf0f839e49 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -108,4 +108,21 @@ boolean isEmpty() { return keyFrequency().isEmpty(); } } + + boolean isValid() { + if (type == StatisticsType.Sketch) { + if (null == keySamples) { + return false; + } + } else { + if (null == keyFrequency()) { + return false; + } + if (keyFrequency().values().contains(null)) { + return false; + } + } + + return true; + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java index 1ac0e386a011..48c85a9bd91e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -48,6 +48,18 @@ class CompletedStatisticsSerializer extends TypeSerializer this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); } + public void changeSortKeySerializerVersion(int version) { + if (sortKeySerializer instanceof SortKeySerializer) { + ((SortKeySerializer) sortKeySerializer).setVersion(version); + } + } + + public void changeSortKeySerializerVersionLatest() { + if (sortKeySerializer instanceof SortKeySerializer) { + ((SortKeySerializer) sortKeySerializer).restoreToLatestVersion(); + } + } + @Override public boolean isImmutableType() { return false; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index e2a282efd82e..4f2afd60fed1 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -370,7 +370,8 @@ public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); this.completedStatistics = StatisticsUtil.deserializeCompletedStatistics( - checkpointData, completedStatisticsSerializer); + checkpointData, (CompletedStatisticsSerializer) completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed this.globalStatistics = globalStatistics( diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index b3e536bdde52..5b979e546d51 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -52,9 +52,12 @@ class SortKeySerializer extends TypeSerializer { private final int size; private final Types.NestedField[] transformedFields; + private int version; + private transient SortKey sortKey; - SortKeySerializer(Schema schema, SortOrder sortOrder) { + SortKeySerializer(Schema schema, SortOrder sortOrder, int version) { + this.version = version; this.schema = schema; this.sortOrder = sortOrder; this.size = sortOrder.fields().size(); @@ -75,6 +78,10 @@ class SortKeySerializer extends TypeSerializer { } } + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this(schema, sortOrder, SortKeySerializerSnapshot.CURRENT_VERSION); + } + private SortKey lazySortKey() { if (sortKey == null) { this.sortKey = new SortKey(schema, sortOrder); @@ -83,6 +90,18 @@ private SortKey lazySortKey() { return sortKey; } + public int getLatestVersion() { + return snapshotConfiguration().getCurrentVersion(); + } + + public void restoreToLatestVersion() { + this.version = snapshotConfiguration().getCurrentVersion(); + } + + public void setVersion(int version) { + this.version = version; + } + @Override public boolean isImmutableType() { return false; @@ -124,6 +143,16 @@ public void serialize(SortKey record, DataOutputView target) throws IOException for (int i = 0; i < size; ++i) { int fieldId = transformedFields[i].fieldId(); Type.TypeID typeId = transformedFields[i].type().typeId(); + if (version > 1) { + Object value = record.get(i, Object.class); + if (value == null) { + target.writeBoolean(true); + continue; + } else { + target.writeBoolean(false); + } + } + switch (typeId) { case BOOLEAN: target.writeBoolean(record.get(i, Boolean.class)); @@ -192,6 +221,14 @@ public SortKey deserialize(SortKey reuse, DataInputView source) throws IOExcepti reuse.size(), size); for (int i = 0; i < size; ++i) { + if (version > 1) { + boolean isNull = source.readBoolean(); + if (isNull) { + reuse.set(i, null); + continue; + } + } + int fieldId = transformedFields[i].fieldId(); Type.TypeID typeId = transformedFields[i].type().typeId(); switch (typeId) { @@ -276,11 +313,13 @@ public TypeSerializerSnapshot snapshotConfiguration() { } public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { - private static final int CURRENT_VERSION = 1; + private static final int CURRENT_VERSION = 2; private Schema schema; private SortOrder sortOrder; + private int version = CURRENT_VERSION; + /** Constructor for read instantiation. */ @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public SortKeySerializerSnapshot() { @@ -310,10 +349,16 @@ public void writeSnapshot(DataOutputView out) throws IOException { @Override public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { - if (readVersion == 1) { - readV1(in); - } else { - throw new IllegalArgumentException("Unknown read version: " + readVersion); + switch (readVersion) { + case 1: + read(in); + this.version = 1; + break; + case 2: + read(in); + break; + default: + throw new IllegalArgumentException("Unknown read version: " + readVersion); } } @@ -324,6 +369,10 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } + if (oldSerializerSnapshot.getCurrentVersion() == 1 && this.getCurrentVersion() == 2) { + return TypeSerializerSchemaCompatibility.compatibleAfterMigration(); + } + // Sort order should be identical SortKeySerializerSnapshot oldSnapshot = (SortKeySerializerSnapshot) oldSerializerSnapshot; if (!sortOrder.sameOrder(oldSnapshot.sortOrder)) { @@ -349,10 +398,10 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( public TypeSerializer restoreSerializer() { Preconditions.checkState(schema != null, "Invalid schema: null"); Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); - return new SortKeySerializer(schema, sortOrder); + return new SortKeySerializer(schema, sortOrder, version); } - private void readV1(DataInputView in) throws IOException { + private void read(DataInputView in) throws IOException { String schemaJson = StringUtils.readString(in); String sortOrderJson = StringUtils.readString(in); this.schema = SchemaParser.fromJson(schemaJson); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java index 5d48ec57ca49..f2efc7fa9834 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java @@ -73,12 +73,29 @@ static byte[] serializeCompletedStatistics( } static CompletedStatistics deserializeCompletedStatistics( - byte[] bytes, TypeSerializer statisticsSerializer) { + byte[] bytes, CompletedStatisticsSerializer statisticsSerializer) { try { DataInputDeserializer input = new DataInputDeserializer(bytes); - return statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + CompletedStatistics completedStatistics = statisticsSerializer.deserialize(input); + if (!completedStatistics.isValid()) { + throw new RuntimeException("Fail to deserialize aggregated statistics,change to v1"); + } + + return completedStatistics; + } catch (Exception e) { + try { + // If we restore from a lower version, the new version of SortKeySerializer cannot correctly + // parse the checkpointData, so we need to first switch the version to v1. Once the state + // data is successfully parsed, we need to switch the serialization version to the latest + // version to parse the subsequent data passed from the TM. + statisticsSerializer.changeSortKeySerializerVersion(1); + DataInputDeserializer input = new DataInputDeserializer(bytes); + CompletedStatistics deserialize = statisticsSerializer.deserialize(input); + statisticsSerializer.changeSortKeySerializerVersionLatest(); + return deserialize; + } catch (IOException ioException) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", ioException); + } } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java index df8c3c79d3e3..aa9a0291b38f 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -46,6 +46,7 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; 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.Maps; @@ -252,6 +253,44 @@ public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); } + @TestTemplate + public void testRangeDistributionWithNullValue() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + List> charRows = createCharRows(numOfCheckpoints, 10); + charRows.add(ImmutableList.of(Row.of(1, null))); + DataStream dataStream = + env.addSource(createRangeDistributionBoundedSource(charRows), ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // sort based on partition columns + builder.append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + } + @TestTemplate public void testRangeDistributionWithSortOrder() throws Exception { table diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java index 4ee9888934a8..1975d7e8d654 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -19,11 +19,15 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; public class TestCompletedStatisticsSerializer extends SerializerTestBase { @@ -51,4 +55,49 @@ protected CompletedStatistics[] getTestData() { CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) }; } + + @Test + public void testSerializer() throws Exception { + TypeSerializer completedStatisticsTypeSerializer = createSerializer(); + CompletedStatistics[] data = getTestData(); + DataOutputSerializer output = new DataOutputSerializer(1024); + completedStatisticsTypeSerializer.serialize(data[0], output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + CompletedStatistics deserialized = completedStatisticsTypeSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(data[0]); + } + + @Test + public void testRestoreOldVersionSerializer() throws Exception { + CompletedStatisticsSerializer completedStatisticsTypeSerializer = + (CompletedStatisticsSerializer) createSerializer(); + completedStatisticsTypeSerializer.changeSortKeySerializerVersion(1); + CompletedStatistics[] data = getTestData(); + DataOutputSerializer output = new DataOutputSerializer(1024); + completedStatisticsTypeSerializer.serialize(data[0], output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + completedStatisticsTypeSerializer.changeSortKeySerializerVersionLatest(); + CompletedStatistics completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + serializedBytes, completedStatisticsTypeSerializer); + assertThat(completedStatistics).isEqualTo(data[0]); + } + + @Test + public void testRestoreNewSerializer() throws Exception { + CompletedStatisticsSerializer completedStatisticsTypeSerializer = + (CompletedStatisticsSerializer) createSerializer(); + CompletedStatistics[] data = getTestData(); + DataOutputSerializer output = new DataOutputSerializer(1024); + completedStatisticsTypeSerializer.serialize(data[0], output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + CompletedStatistics completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + serializedBytes, completedStatisticsTypeSerializer); + assertThat(completedStatistics).isEqualTo(data[0]); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index a08a8a73e80c..acecc5b60af1 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -152,6 +152,80 @@ public void testDataStatisticsEventHandling(StatisticsType type) throws Exceptio } } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testDataStatisticsEventHandlingWithNullValue(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + SortKey nullSortKey = Fixtures.SORT_KEY.copy(); + nullSortKey.set(0, null); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + nullSortKey, + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + nullSortKey, + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + Map keyFrequency = + ImmutableMap.of(nullSortKey, 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 5L); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(NUM_SUBTASKS, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(keyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + nullSortKey, + nullSortKey, + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.checkpointId()).isEqualTo(1L); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + } + } + } + @Test public void testRequestGlobalStatisticsEventHandling() throws Exception { try (DataStatisticsCoordinator dataStatisticsCoordinator = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index bc248b778184..f7a7a147e73a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -34,6 +34,8 @@ import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; @@ -136,6 +138,34 @@ public void testProcessElement(StatisticsType type) throws Exception { } } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testProcessElementWithNull(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + operator.processElement(new StreamRecord<>(GenericRowData.of(null, 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + + DataStatistics localStatistics = operator.localStatistics(); + SortKeySerializer sortKeySerializer = + new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); + DataStatisticsSerializer taskStatisticsSerializer = + new DataStatisticsSerializer(sortKeySerializer); + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + taskStatisticsSerializer.serialize(localStatistics, outputView); + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + DataStatistics dataStatistics = taskStatisticsSerializer.deserialize(inputView); + + testHarness.endInput(); + + assertThat(localStatistics).isEqualTo(dataStatistics); + } + } + @ParameterizedTest @EnumSource(StatisticsType.class) public void testOperatorOutput(StatisticsType type) throws Exception { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java index 54cceae6e55b..ac2e2784e681 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -80,8 +80,8 @@ public void testSerializationSize() throws Exception { byte[] serializedBytes = output.getCopyOfBuffer(); assertThat(serializedBytes.length) .as( - "Serialized bytes for sort key should be 38 bytes (34 UUID text + 4 byte integer of string length") - .isEqualTo(38); + "Serialized bytes for sort key should be 39 bytes (34 UUID text + 4 byte integer of string length + 1 byte of isnull flag") + .isEqualTo(39); DataInputDeserializer input = new DataInputDeserializer(serializedBytes); SortKey deserialized = serializer.deserialize(input); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java index c0f688f2589e..2d87b089cecb 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -73,6 +73,28 @@ public void testRestoredSerializer() throws Exception { assertThat(deserialized).isEqualTo(sortKey); } + @Test + public void testRestoredOldSerializer() throws Exception { + RowData rowData = GenericRowData.of(StringData.fromString("str"), 1); + RowDataWrapper rowDataWrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(struct); + + SortKeySerializer originalSerializer = new SortKeySerializer(SCHEMA, SORT_ORDER, 1); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + ((SortKeySerializer) restoredSerializer).setVersion(1); + DataOutputSerializer output = new DataOutputSerializer(1024); + originalSerializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = restoredSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } + @Test public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = From c0d6d42a55a7d33d7d38f73d00e46e86b57d934e Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Fri, 3 Jan 2025 09:40:04 -0700 Subject: [PATCH 310/313] Spark: Change delete file granularity to file in Spark 3.5 (#11478) --- .../iceberg/spark/extensions/TestDelete.java | 10 ++++++---- .../apache/iceberg/spark/extensions/TestMerge.java | 10 ++++++++-- .../TestRewritePositionDeleteFilesProcedure.java | 2 +- .../iceberg/spark/extensions/TestUpdate.java | 10 ++++++---- .../org/apache/iceberg/spark/SparkWriteConf.java | 14 ++++++-------- .../apache/iceberg/spark/TestSparkWriteConf.java | 6 +++--- .../TestRewritePositionDeleteFilesAction.java | 4 +++- 7 files changed, 33 insertions(+), 23 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index 5e42c2dabb41..08d1b7ccbbcf 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -63,6 +63,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -151,12 +152,15 @@ public void testCoalesceDelete() throws Exception { // set the open file cost large enough to produce a separate scan task per file // use range distribution to trigger a shuffle + // set partitioned scoped deletes so that 1 delete file is written as part of the output task Map tableProps = ImmutableMap.of( SPLIT_OPEN_FILE_COST, String.valueOf(Integer.MAX_VALUE), DELETE_DISTRIBUTION_MODE, - DistributionMode.RANGE.modeName()); + DistributionMode.RANGE.modeName(), + TableProperties.DELETE_GRANULARITY, + DeleteGranularity.PARTITION.toString()); sql("ALTER TABLE %s SET TBLPROPERTIES (%s)", tableName, tablePropsAsString(tableProps)); createBranchIfNeeded(); @@ -1306,10 +1310,8 @@ public void testDeleteWithMultipleSpecs() { Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { validateCopyOnWrite(currentSnapshot, "3", "4", "1"); - } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { - validateMergeOnRead(currentSnapshot, "3", "4", null); } else { - validateMergeOnRead(currentSnapshot, "3", "3", null); + validateMergeOnRead(currentSnapshot, "3", "4", null); } assertEquals( diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java index 5304e6f752df..11bd6bebd66d 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; import static org.apache.iceberg.TableProperties.MERGE_DISTRIBUTION_MODE; @@ -56,6 +57,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -233,7 +235,6 @@ public void testMergeWithVectorizedReads() { @TestTemplate public void testCoalesceMerge() { - assumeThat(formatVersion).isLessThan(3); createAndInitTable("id INT, salary INT, dep STRING"); String[] records = new String[100]; @@ -252,7 +253,9 @@ public void testCoalesceMerge() { SPLIT_OPEN_FILE_COST, String.valueOf(Integer.MAX_VALUE), MERGE_DISTRIBUTION_MODE, - DistributionMode.NONE.modeName()); + DistributionMode.NONE.modeName(), + TableProperties.DELETE_GRANULARITY, + DeleteGranularity.PARTITION.toString()); sql("ALTER TABLE %s SET TBLPROPERTIES (%s)", tableName, tablePropsAsString(tableProps)); createBranchIfNeeded(); @@ -295,6 +298,9 @@ public void testCoalesceMerge() { // AQE detects that all shuffle blocks are small and processes them in 1 task // otherwise, there would be 200 tasks writing to the table validateProperty(currentSnapshot, SnapshotSummary.ADDED_FILES_PROP, "1"); + } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { + validateProperty(currentSnapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); + validateProperty(currentSnapshot, SnapshotSummary.ADDED_DVS_PROP, "4"); } else { // MoR MERGE would perform a join on `id` // every task has data for each of 200 reducers diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index bb82b63d208d..eae9208022a1 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -49,7 +49,7 @@ private void createTable(boolean partitioned) throws Exception { String partitionStmt = partitioned ? "PARTITIONED BY (id)" : ""; sql( "CREATE TABLE %s (id bigint, data string) USING iceberg %s TBLPROPERTIES" - + "('format-version'='2', 'write.delete.mode'='merge-on-read')", + + "('format-version'='2', 'write.delete.mode'='merge-on-read', 'write.delete.granularity'='partition')", tableName, partitionStmt); List records = diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java index 6332e303ad67..b28ea0e286e9 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -58,6 +58,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -131,12 +132,15 @@ public void testCoalesceUpdate() { // set the open file cost large enough to produce a separate scan task per file // use range distribution to trigger a shuffle + // set partitioned scoped deletes so that 1 delete file is written as part of the output task Map tableProps = ImmutableMap.of( SPLIT_OPEN_FILE_COST, String.valueOf(Integer.MAX_VALUE), UPDATE_DISTRIBUTION_MODE, - DistributionMode.RANGE.modeName()); + DistributionMode.RANGE.modeName(), + TableProperties.DELETE_GRANULARITY, + DeleteGranularity.PARTITION.toString()); sql("ALTER TABLE %s SET TBLPROPERTIES (%s)", tableName, tablePropsAsString(tableProps)); createBranchIfNeeded(); @@ -440,10 +444,8 @@ public void testUpdateWithoutCondition() { validateProperty(currentSnapshot, CHANGED_PARTITION_COUNT_PROP, "2"); validateProperty(currentSnapshot, DELETED_FILES_PROP, "3"); validateProperty(currentSnapshot, ADDED_FILES_PROP, ImmutableSet.of("2", "3")); - } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { - validateMergeOnRead(currentSnapshot, "2", "3", "2"); } else { - validateMergeOnRead(currentSnapshot, "2", "2", "2"); + validateMergeOnRead(currentSnapshot, "2", "3", "2"); } assertEquals( diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java index 7ba8d558d5e7..f9fb904db394 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java @@ -714,14 +714,12 @@ private double shuffleCompressionRatio(FileFormat outputFileFormat, String outpu } public DeleteGranularity deleteGranularity() { - String valueAsString = - confParser - .stringConf() - .option(SparkWriteOptions.DELETE_GRANULARITY) - .tableProperty(TableProperties.DELETE_GRANULARITY) - .defaultValue(TableProperties.DELETE_GRANULARITY_DEFAULT) - .parse(); - return DeleteGranularity.fromString(valueAsString); + return confParser + .enumConf(DeleteGranularity::fromString) + .option(SparkWriteOptions.DELETE_GRANULARITY) + .tableProperty(TableProperties.DELETE_GRANULARITY) + .defaultValue(DeleteGranularity.FILE) + .parse(); } public boolean useDVs() { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java index c2df62697882..42d697410377 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java @@ -142,7 +142,7 @@ public void testDeleteGranularityDefault() { SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); DeleteGranularity value = writeConf.deleteGranularity(); - assertThat(value).isEqualTo(DeleteGranularity.PARTITION); + assertThat(value).isEqualTo(DeleteGranularity.FILE); } @TestTemplate @@ -151,13 +151,13 @@ public void testDeleteGranularityTableProperty() { table .updateProperties() - .set(TableProperties.DELETE_GRANULARITY, DeleteGranularity.FILE.toString()) + .set(TableProperties.DELETE_GRANULARITY, DeleteGranularity.PARTITION.toString()) .commit(); SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); DeleteGranularity value = writeConf.deleteGranularity(); - assertThat(value).isEqualTo(DeleteGranularity.FILE); + assertThat(value).isEqualTo(DeleteGranularity.PARTITION); } @TestTemplate diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 8ecec5ac2d42..a2da5cc447df 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -757,7 +757,9 @@ private Map tableProperties() { TableProperties.FORMAT_VERSION, "2", TableProperties.DEFAULT_FILE_FORMAT, - format.toString()); + format.toString(), + TableProperties.DELETE_GRANULARITY, + DeleteGranularity.PARTITION.toString()); } private void writeRecords(Table table, int files, int numRecords) { From dbfefb07312be8554438c1f16f1037ab22bf153b Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 4 Jan 2025 04:00:33 +0800 Subject: [PATCH 311/313] Bump Apache Spark to 3.5.4 (#11731) --- gradle/libs.versions.toml | 2 +- .../spark/data/vectorized/IcebergArrowColumnVector.java | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 556187ceecb8..d10bb876afdf 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -82,7 +82,7 @@ slf4j = "2.0.16" snowflake-jdbc = "3.20.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.4" -spark-hive35 = "3.5.2" +spark-hive35 = "3.5.4" sqlite-jdbc = "3.47.1.0" testcontainers = "1.20.4" tez010 = "0.10.4" diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java index 38ec3a0e838c..dc118aebe374 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java @@ -59,6 +59,11 @@ public void close() { accessor.close(); } + public void closeIfFreeable() { + // If a column vector is writable or constant, it should override this method and do nothing. + // See more details at SPARK-50235, SPARK-50463 (Fixed in Spark 3.5.4) + } + @Override public boolean hasNull() { return nullabilityHolder.hasNulls(); From fcd5dd932a21066d6127c94c50f3de43e8c2d80c Mon Sep 17 00:00:00 2001 From: Vova Kolmakov Date: Sun, 5 Jan 2025 01:25:10 +0700 Subject: [PATCH 312/313] Kafka-connect-runtime: remove code duplications in integration tests (#11883) Co-authored-by: Vova Kolmakov --- .../connect/IntegrationDynamicTableTest.java | 78 +++-------------- .../connect/IntegrationMultiTableTest.java | 87 ++++--------------- .../iceberg/connect/IntegrationTest.java | 84 ++++-------------- .../iceberg/connect/IntegrationTestBase.java | 74 +++++++++++++++- 4 files changed, 123 insertions(+), 200 deletions(-) diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationDynamicTableTest.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationDynamicTableTest.java index 5c458ad3fa78..1603b8927e62 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationDynamicTableTest.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationDynamicTableTest.java @@ -20,44 +20,22 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.time.Duration; import java.time.Instant; import java.util.List; import org.apache.iceberg.DataFile; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.NullSource; import org.junit.jupiter.params.provider.ValueSource; public class IntegrationDynamicTableTest extends IntegrationTestBase { - private static final String TEST_DB = "test"; private static final String TEST_TABLE1 = "tbl1"; private static final String TEST_TABLE2 = "tbl2"; private static final TableIdentifier TABLE_IDENTIFIER1 = TableIdentifier.of(TEST_DB, TEST_TABLE1); private static final TableIdentifier TABLE_IDENTIFIER2 = TableIdentifier.of(TEST_DB, TEST_TABLE2); - @BeforeEach - public void before() { - createTopic(testTopic(), TEST_TOPIC_PARTITIONS); - ((SupportsNamespaces) catalog()).createNamespace(Namespace.of(TEST_DB)); - } - - @AfterEach - public void after() { - context().stopConnector(connectorName()); - deleteTopic(testTopic()); - catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE1)); - catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE2)); - ((SupportsNamespaces) catalog()).dropNamespace(Namespace.of(TEST_DB)); - } - @ParameterizedTest @NullSource @ValueSource(strings = "test_branch") @@ -68,7 +46,7 @@ public void testIcebergSink(String branch) { catalog().createTable(TABLE_IDENTIFIER2, TestEvent.TEST_SCHEMA); boolean useSchema = branch == null; // use a schema for one of the tests - runTest(branch, useSchema); + runTest(branch, useSchema, ImmutableMap.of(), List.of(TABLE_IDENTIFIER1, TABLE_IDENTIFIER2)); List files = dataFiles(TABLE_IDENTIFIER1, branch); assertThat(files).hasSize(1); @@ -81,36 +59,15 @@ public void testIcebergSink(String branch) { assertSnapshotProps(TABLE_IDENTIFIER2, branch); } - private void runTest(String branch, boolean useSchema) { - // set offset reset to earliest so we don't miss any test messages - KafkaConnectUtils.Config connectorConfig = - new KafkaConnectUtils.Config(connectorName()) - .config("topics", testTopic()) - .config("connector.class", IcebergSinkConnector.class.getName()) - .config("tasks.max", 2) - .config("consumer.override.auto.offset.reset", "earliest") - .config("key.converter", "org.apache.kafka.connect.json.JsonConverter") - .config("key.converter.schemas.enable", false) - .config("value.converter", "org.apache.kafka.connect.json.JsonConverter") - .config("value.converter.schemas.enable", useSchema) - .config("iceberg.tables.dynamic-enabled", true) - .config("iceberg.tables.route-field", "payload") - .config("iceberg.control.commit.interval-ms", 1000) - .config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE) - .config("iceberg.kafka.auto.offset.reset", "earliest"); - - context().connectorCatalogProperties().forEach(connectorConfig::config); - - if (branch != null) { - connectorConfig.config("iceberg.tables.default-commit-branch", branch); - } - - if (!useSchema) { - connectorConfig.config("value.converter.schemas.enable", false); - } - - context().startConnector(connectorConfig); + @Override + protected KafkaConnectUtils.Config createConfig(boolean useSchema) { + return createCommonConfig(useSchema) + .config("iceberg.tables.dynamic-enabled", true) + .config("iceberg.tables.route-field", "payload"); + } + @Override + protected void sendEvents(boolean useSchema) { TestEvent event1 = new TestEvent(1, "type1", Instant.now(), TEST_DB + "." + TEST_TABLE1); TestEvent event2 = new TestEvent(2, "type2", Instant.now(), TEST_DB + "." + TEST_TABLE2); TestEvent event3 = new TestEvent(3, "type3", Instant.now(), TEST_DB + ".tbl3"); @@ -118,18 +75,11 @@ private void runTest(String branch, boolean useSchema) { send(testTopic(), event1, useSchema); send(testTopic(), event2, useSchema); send(testTopic(), event3, useSchema); - flush(); - - Awaitility.await() - .atMost(Duration.ofSeconds(30)) - .pollInterval(Duration.ofSeconds(1)) - .untilAsserted(this::assertSnapshotAdded); } - private void assertSnapshotAdded() { - Table table = catalog().loadTable(TABLE_IDENTIFIER1); - assertThat(table.snapshots()).hasSize(1); - table = catalog().loadTable(TABLE_IDENTIFIER2); - assertThat(table.snapshots()).hasSize(1); + @Override + void dropTables() { + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE1)); + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE2)); } } diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationMultiTableTest.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationMultiTableTest.java index 7cffbd8838b2..d85514c3f92d 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationMultiTableTest.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationMultiTableTest.java @@ -20,44 +20,22 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.time.Duration; import java.time.Instant; import java.util.List; import org.apache.iceberg.DataFile; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.NullSource; import org.junit.jupiter.params.provider.ValueSource; public class IntegrationMultiTableTest extends IntegrationTestBase { - private static final String TEST_DB = "test"; private static final String TEST_TABLE1 = "foobar1"; private static final String TEST_TABLE2 = "foobar2"; private static final TableIdentifier TABLE_IDENTIFIER1 = TableIdentifier.of(TEST_DB, TEST_TABLE1); private static final TableIdentifier TABLE_IDENTIFIER2 = TableIdentifier.of(TEST_DB, TEST_TABLE2); - @BeforeEach - public void before() { - createTopic(testTopic(), TEST_TOPIC_PARTITIONS); - ((SupportsNamespaces) catalog()).createNamespace(Namespace.of(TEST_DB)); - } - - @AfterEach - public void after() { - context().stopConnector(connectorName()); - deleteTopic(testTopic()); - catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE1)); - catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE2)); - ((SupportsNamespaces) catalog()).dropNamespace(Namespace.of(TEST_DB)); - } - @ParameterizedTest @NullSource @ValueSource(strings = "test_branch") @@ -68,7 +46,7 @@ public void testIcebergSink(String branch) { catalog().createTable(TABLE_IDENTIFIER2, TestEvent.TEST_SCHEMA); boolean useSchema = branch == null; // use a schema for one of the tests - runTest(branch, useSchema); + runTest(branch, useSchema, ImmutableMap.of(), List.of(TABLE_IDENTIFIER1, TABLE_IDENTIFIER2)); List files = dataFiles(TABLE_IDENTIFIER1, branch); assertThat(files).hasSize(1); @@ -81,41 +59,19 @@ public void testIcebergSink(String branch) { assertSnapshotProps(TABLE_IDENTIFIER2, branch); } - private void runTest(String branch, boolean useSchema) { - // set offset reset to earliest so we don't miss any test messages - KafkaConnectUtils.Config connectorConfig = - new KafkaConnectUtils.Config(connectorName()) - .config("topics", testTopic()) - .config("connector.class", IcebergSinkConnector.class.getName()) - .config("tasks.max", 2) - .config("consumer.override.auto.offset.reset", "earliest") - .config("key.converter", "org.apache.kafka.connect.json.JsonConverter") - .config("key.converter.schemas.enable", false) - .config("value.converter", "org.apache.kafka.connect.json.JsonConverter") - .config("value.converter.schemas.enable", useSchema) - .config( - "iceberg.tables", - String.format("%s.%s, %s.%s", TEST_DB, TEST_TABLE1, TEST_DB, TEST_TABLE2)) - .config("iceberg.tables.route-field", "type") - .config(String.format("iceberg.table.%s.%s.route-regex", TEST_DB, TEST_TABLE1), "type1") - .config(String.format("iceberg.table.%s.%s.route-regex", TEST_DB, TEST_TABLE2), "type2") - .config("iceberg.control.commit.interval-ms", 1000) - .config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE) - .config("iceberg.kafka.auto.offset.reset", "earliest"); - - context().connectorCatalogProperties().forEach(connectorConfig::config); - - if (branch != null) { - connectorConfig.config("iceberg.tables.default-commit-branch", branch); - } - - // use a schema for one of the cases - if (!useSchema) { - connectorConfig.config("value.converter.schemas.enable", false); - } - - context().startConnector(connectorConfig); + @Override + protected KafkaConnectUtils.Config createConfig(boolean useSchema) { + return createCommonConfig(useSchema) + .config( + "iceberg.tables", + String.format("%s.%s, %s.%s", TEST_DB, TEST_TABLE1, TEST_DB, TEST_TABLE2)) + .config("iceberg.tables.route-field", "type") + .config(String.format("iceberg.table.%s.%s.route-regex", TEST_DB, TEST_TABLE1), "type1") + .config(String.format("iceberg.table.%s.%s.route-regex", TEST_DB, TEST_TABLE2), "type2"); + } + @Override + protected void sendEvents(boolean useSchema) { TestEvent event1 = new TestEvent(1, "type1", Instant.now(), "hello world!"); TestEvent event2 = new TestEvent(2, "type2", Instant.now(), "having fun?"); TestEvent event3 = new TestEvent(3, "type3", Instant.now(), "ignore me"); @@ -123,18 +79,11 @@ private void runTest(String branch, boolean useSchema) { send(testTopic(), event1, useSchema); send(testTopic(), event2, useSchema); send(testTopic(), event3, useSchema); - flush(); - - Awaitility.await() - .atMost(Duration.ofSeconds(30)) - .pollInterval(Duration.ofSeconds(1)) - .untilAsserted(this::assertSnapshotAdded); } - private void assertSnapshotAdded() { - Table table = catalog().loadTable(TABLE_IDENTIFIER1); - assertThat(table.snapshots()).hasSize(1); - table = catalog().loadTable(TABLE_IDENTIFIER2); - assertThat(table.snapshots()).hasSize(1); + @Override + void dropTables() { + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE1)); + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE2)); } } diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTest.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTest.java index 80a74539311c..71a306e310bb 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTest.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTest.java @@ -19,7 +19,6 @@ package org.apache.iceberg.connect; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; import java.time.Duration; import java.time.Instant; @@ -28,11 +27,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -41,33 +36,15 @@ import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.types.Types.StringType; import org.apache.iceberg.types.Types.TimestampType; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.NullSource; import org.junit.jupiter.params.provider.ValueSource; public class IntegrationTest extends IntegrationTestBase { - private static final String TEST_DB = "test"; private static final String TEST_TABLE = "foobar"; private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(TEST_DB, TEST_TABLE); - @BeforeEach - public void before() { - createTopic(testTopic(), TEST_TOPIC_PARTITIONS); - ((SupportsNamespaces) catalog()).createNamespace(Namespace.of(TEST_DB)); - } - - @AfterEach - public void after() { - context().stopConnector(connectorName()); - deleteTopic(testTopic()); - catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE)); - ((SupportsNamespaces) catalog()).dropNamespace(Namespace.of(TEST_DB)); - } - @ParameterizedTest @NullSource @ValueSource(strings = "test_branch") @@ -75,7 +52,7 @@ public void testIcebergSinkPartitionedTable(String branch) { catalog().createTable(TABLE_IDENTIFIER, TestEvent.TEST_SCHEMA, TestEvent.TEST_SPEC); boolean useSchema = branch == null; // use a schema for one of the tests - runTest(branch, useSchema, ImmutableMap.of()); + runTest(branch, useSchema, ImmutableMap.of(), List.of(TABLE_IDENTIFIER)); List files = dataFiles(TABLE_IDENTIFIER, branch); // partition may involve 1 or 2 workers @@ -92,7 +69,7 @@ public void testIcebergSinkUnpartitionedTable(String branch) { catalog().createTable(TABLE_IDENTIFIER, TestEvent.TEST_SCHEMA); boolean useSchema = branch == null; // use a schema for one of the tests - runTest(branch, useSchema, ImmutableMap.of()); + runTest(branch, useSchema, ImmutableMap.of(), List.of(TABLE_IDENTIFIER)); List files = dataFiles(TABLE_IDENTIFIER, branch); // may involve 1 or 2 workers @@ -113,7 +90,11 @@ public void testIcebergSinkSchemaEvolution(String branch) { catalog().createTable(TABLE_IDENTIFIER, initialSchema); boolean useSchema = branch == null; // use a schema for one of the tests - runTest(branch, useSchema, ImmutableMap.of("iceberg.tables.evolve-schema-enabled", "true")); + runTest( + branch, + useSchema, + ImmutableMap.of("iceberg.tables.evolve-schema-enabled", "true"), + List.of(TABLE_IDENTIFIER)); List files = dataFiles(TABLE_IDENTIFIER, branch); // may involve 1 or 2 workers @@ -141,7 +122,7 @@ public void testIcebergSinkAutoCreate(String branch) { extraConfig.put("iceberg.tables.default-partition-by", "hour(ts)"); } - runTest(branch, useSchema, extraConfig); + runTest(branch, useSchema, extraConfig, List.of(TABLE_IDENTIFIER)); List files = dataFiles(TABLE_IDENTIFIER, branch); // may involve 1 or 2 workers @@ -172,33 +153,14 @@ private void assertGeneratedSchema(boolean useSchema, Class expe } } - private void runTest(String branch, boolean useSchema, Map extraConfig) { - // set offset reset to earliest so we don't miss any test messages - KafkaConnectUtils.Config connectorConfig = - new KafkaConnectUtils.Config(connectorName()) - .config("topics", testTopic()) - .config("connector.class", IcebergSinkConnector.class.getName()) - .config("tasks.max", 2) - .config("consumer.override.auto.offset.reset", "earliest") - .config("key.converter", "org.apache.kafka.connect.json.JsonConverter") - .config("key.converter.schemas.enable", false) - .config("value.converter", "org.apache.kafka.connect.json.JsonConverter") - .config("value.converter.schemas.enable", useSchema) - .config("iceberg.tables", String.format("%s.%s", TEST_DB, TEST_TABLE)) - .config("iceberg.control.commit.interval-ms", 1000) - .config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE) - .config("iceberg.kafka.auto.offset.reset", "earliest"); - - context().connectorCatalogProperties().forEach(connectorConfig::config); - - if (branch != null) { - connectorConfig.config("iceberg.tables.default-commit-branch", branch); - } - - extraConfig.forEach(connectorConfig::config); - - context().startConnector(connectorConfig); + @Override + protected KafkaConnectUtils.Config createConfig(boolean useSchema) { + return createCommonConfig(useSchema) + .config("iceberg.tables", String.format("%s.%s", TEST_DB, TEST_TABLE)); + } + @Override + protected void sendEvents(boolean useSchema) { TestEvent event1 = new TestEvent(1, "type1", Instant.now(), "hello world!"); Instant threeDaysAgo = Instant.now().minus(Duration.ofDays(3)); @@ -206,20 +168,10 @@ private void runTest(String branch, boolean useSchema, Map extra send(testTopic(), event1, useSchema); send(testTopic(), event2, useSchema); - flush(); - - Awaitility.await() - .atMost(Duration.ofSeconds(30)) - .pollInterval(Duration.ofSeconds(1)) - .untilAsserted(this::assertSnapshotAdded); } - private void assertSnapshotAdded() { - try { - Table table = catalog().loadTable(TABLE_IDENTIFIER); - assertThat(table.snapshots()).hasSize(1); - } catch (NoSuchTableException e) { - fail("Table should exist"); - } + @Override + void dropTables() { + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE)); } } diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java index 247211edb01f..532cbde6d315 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java @@ -19,7 +19,9 @@ package org.apache.iceberg.connect; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; +import java.time.Duration; import java.util.List; import java.util.Map; import java.util.UUID; @@ -31,7 +33,10 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.kafka.clients.admin.Admin; @@ -39,11 +44,12 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.assertj.core.api.Condition; +import org.awaitility.Awaitility; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; -public class IntegrationTestBase { +public abstract class IntegrationTestBase { private static TestContext context; @@ -55,6 +61,13 @@ public class IntegrationTestBase { private KafkaProducer producer; protected static final int TEST_TOPIC_PARTITIONS = 2; + protected static final String TEST_DB = "test"; + + abstract KafkaConnectUtils.Config createConfig(boolean useSchema); + + abstract void sendEvents(boolean useSchema); + + abstract void dropTables(); protected TestContext context() { return context; @@ -84,10 +97,16 @@ public void baseBefore() { this.admin = context.initLocalAdmin(); this.connectorName = "test_connector-" + UUID.randomUUID(); this.testTopic = "test-topic-" + UUID.randomUUID(); + createTopic(testTopic(), TEST_TOPIC_PARTITIONS); + ((SupportsNamespaces) catalog()).createNamespace(Namespace.of(TEST_DB)); } @AfterEach public void baseAfter() { + context().stopConnector(connectorName()); + deleteTopic(testTopic()); + dropTables(); + ((SupportsNamespaces) catalog()).dropNamespace(Namespace.of(TEST_DB)); try { if (catalog instanceof AutoCloseable) { ((AutoCloseable) catalog).close(); @@ -158,4 +177,57 @@ protected void send(String topicName, TestEvent event, boolean useSchema) { protected void flush() { producer.flush(); } + + protected KafkaConnectUtils.Config createCommonConfig(boolean useSchema) { + // set offset reset to the earliest, so we don't miss any test messages + return new KafkaConnectUtils.Config(connectorName()) + .config("topics", testTopic()) + .config("connector.class", IcebergSinkConnector.class.getName()) + .config("tasks.max", 2) + .config("consumer.override.auto.offset.reset", "earliest") + .config("key.converter", "org.apache.kafka.connect.json.JsonConverter") + .config("key.converter.schemas.enable", false) + .config("value.converter", "org.apache.kafka.connect.json.JsonConverter") + .config("value.converter.schemas.enable", useSchema) + .config("iceberg.control.commit.interval-ms", 1000) + .config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE) + .config("iceberg.kafka.auto.offset.reset", "earliest"); + } + + protected void runTest( + String branch, + boolean useSchema, + Map extraConfig, + List tableIdentifiers) { + KafkaConnectUtils.Config connectorConfig = createConfig(useSchema); + + context().connectorCatalogProperties().forEach(connectorConfig::config); + + if (branch != null) { + connectorConfig.config("iceberg.tables.default-commit-branch", branch); + } + + extraConfig.forEach(connectorConfig::config); + + context().startConnector(connectorConfig); + + sendEvents(useSchema); + flush(); + + Awaitility.await() + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofSeconds(1)) + .untilAsserted(() -> assertSnapshotAdded(tableIdentifiers)); + } + + protected void assertSnapshotAdded(List tableIdentifiers) { + for (TableIdentifier tableId : tableIdentifiers) { + try { + Table table = catalog().loadTable(tableId); + assertThat(table.snapshots()).hasSize(1); + } catch (NoSuchTableException e) { + fail("Table should exist"); + } + } + } } From 2247f59e6717cfb542963d824bcd13680d372409 Mon Sep 17 00:00:00 2001 From: Dylan Nguyen Date: Sun, 5 Jan 2025 01:00:31 -0800 Subject: [PATCH 313/313] Work in progress HashBucketing and Batch Processing --- .../org/apache/iceberg/data/DeleteFilter.java | 92 ++++++++++++++++--- 1 file changed, 79 insertions(+), 13 deletions(-) diff --git a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java index e7d8445cf8c8..e34e3ee14071 100644 --- a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java +++ b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java @@ -17,8 +17,14 @@ * under the License. */ package org.apache.iceberg.data; - +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -44,6 +50,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.StructProjection; +import org.junit.jupiter.api.TestTemplate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,7 +62,7 @@ public abstract class DeleteFilter { private final List eqDeletes; private final Schema requiredSchema; private final Accessor posAccessor; - private final boolean hasIsDeletedColumn; + private final boolean hasIsDeletedClumn; private final int isDeletedColumnPosition; private final DeleteCounter counter; @@ -63,6 +70,8 @@ public abstract class DeleteFilter { private PositionDeleteIndex deleteRowPositions = null; private List> isInDeleteSets = null; private Predicate eqDeleteRows = null; + private final int batchSize = 100; // Number of delete files to process per batch + protected DeleteFilter( String filePath, @@ -149,7 +158,6 @@ private DeleteLoader deleteLoader() { if (deleteLoader == null) { synchronized (this) { if (deleteLoader == null) { - this.deleteLoader = newDeleteLoader(); } } } @@ -161,39 +169,97 @@ public CloseableIterable filter(CloseableIterable records) { return applyEqDeletes(applyPosDeletes(records)); } - private List> applyEqDeletes() { + + // refer to final int batchsize at the top + // this is starter code, need to write a test to explore method further. + public List> applyEqDeletes() { if (isInDeleteSets != null) { return isInDeleteSets; } - isInDeleteSets = Lists.newArrayList(); + isInDeleteSets = new ArrayList<>(); if (eqDeletes.isEmpty()) { return isInDeleteSets; } - Multimap, DeleteFile> filesByDeleteIds = - Multimaps.newMultimap(Maps.newHashMap(), Lists::newArrayList); + List currentBatch = new ArrayList<>(); for (DeleteFile delete : eqDeletes) { + currentBatch.add(delete); + + if (currentBatch.size() >= batchSize) { + processBatchAndAddPredicates(currentBatch); + currentBatch.clear(); + } + } + + if (!currentBatch.isEmpty()) { + processBatchAndAddPredicates(currentBatch); + } + + return isInDeleteSets; + } + + private void processBatchAndAddPredicates(List deleteBatch) { + Multimap, DeleteFile> filesByDeleteIds = + Multimaps.newMultimap(Maps.newHashMap(), ArrayList::new); + + for (DeleteFile delete : deleteBatch) { filesByDeleteIds.put(Sets.newHashSet(delete.equalityFieldIds()), delete); } for (Map.Entry, Collection> entry : - filesByDeleteIds.asMap().entrySet()) { + filesByDeleteIds.asMap().entrySet()) { Set ids = entry.getKey(); Iterable deletes = entry.getValue(); Schema deleteSchema = TypeUtil.select(requiredSchema, ids); - // a projection to select and reorder fields of the file schema to match the delete rows StructProjection projectRow = StructProjection.create(requiredSchema, deleteSchema); + Map> hashBuckets = new HashMap<>(); + for (DeleteFile delete : deletes) { + for (StructLike deleteRecord : deleteLoader().loadEqualityDeletes((Iterable) delete, deleteSchema)) { + StructLike projectedDeleteRecord = projectRow.wrap(deleteRecord); + + int hash = computeHash(projectedDeleteRecord); + + hashBuckets.computeIfAbsent(hash, k -> new ArrayList<>()).add(projectedDeleteRecord); + } + } + + Predicate isInDeleteSet = record -> { + StructLike wrappedRecord = projectRow.wrap(asStructLike(record)); + + int hash = computeHash(wrappedRecord); + + if (!hashBuckets.containsKey(hash)) { + return false; + } + + List deleteRecords = hashBuckets.get(hash); + for (StructLike deleteRecord : deleteRecords) { + if (deleteRecord.equals(wrappedRecord)) { + return true; + } + } + return false; + }; - StructLikeSet deleteSet = deleteLoader().loadEqualityDeletes(deletes, deleteSchema); - Predicate isInDeleteSet = - record -> deleteSet.contains(projectRow.wrap(asStructLike(record))); isInDeleteSets.add(isInDeleteSet); } + } - return isInDeleteSets; + + private int computeHash(StructLike record) { + try { + MessageDigest digest = MessageDigest.getInstance("SHA-1"); + byte[] bytes = record.toString().getBytes(StandardCharsets.UTF_8); + byte[] hashBytes = digest.digest(bytes); + + // Convert the first 4 bytes of the hash into an integer + return ByteBuffer.wrap(hashBytes).getInt(); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException("Error computing hash", e); + } } public CloseableIterable findEqualityDeleteRows(CloseableIterable records) {